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 2014/01/10 03:38:06 UTC

[01/37] git commit: Renaming Client => AppClient

Updated Branches:
  refs/heads/master 26cdb5f68 -> d86a85e9c


Renaming Client => AppClient


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

Branch: refs/heads/master
Commit: 1070b566d4cde4e9a69ccd318747b218f5a44dc7
Parents: 85a344b
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sat Dec 21 19:55:15 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Dec 25 01:17:01 2013 -0800

----------------------------------------------------------------------
 .../apache/spark/deploy/client/AppClient.scala  | 192 +++++++++++++++++++
 .../spark/deploy/client/AppClientListener.scala |  39 ++++
 .../org/apache/spark/deploy/client/Client.scala | 191 ------------------
 .../spark/deploy/client/ClientListener.scala    |  39 ----
 .../apache/spark/deploy/client/TestClient.scala |   4 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |   8 +-
 6 files changed, 237 insertions(+), 236 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1070b566/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
new file mode 100644
index 0000000..c5a0d1f
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
@@ -0,0 +1,192 @@
+/*
+ * 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.client
+
+import java.util.concurrent.TimeoutException
+
+import scala.concurrent.duration._
+import scala.concurrent.Await
+
+import akka.actor._
+import akka.pattern.ask
+import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent}
+
+import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
+import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.master.Master
+import org.apache.spark.util.AkkaUtils
+
+
+/**
+ * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
+ * an app description, and a listener for cluster events, and calls back the listener when various
+ * events occur.
+ *
+ * @param masterUrls Each url should look like spark://host:port.
+ */
+private[spark] class AppClient(
+    actorSystem: ActorSystem,
+    masterUrls: Array[String],
+    appDescription: ApplicationDescription,
+    listener: AppClientListener)
+  extends Logging {
+
+  val REGISTRATION_TIMEOUT = 20.seconds
+  val REGISTRATION_RETRIES = 3
+
+  var masterAddress: Address = null
+  var actor: ActorRef = null
+  var appId: String = null
+  var registered = false
+  var activeMasterUrl: String = null
+
+  class ClientActor extends Actor with Logging {
+    var master: ActorSelection = null
+    var alreadyDisconnected = false  // To avoid calling listener.disconnected() multiple times
+    var alreadyDead = false  // To avoid calling listener.dead() multiple times
+
+    override def preStart() {
+      context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
+      try {
+        registerWithMaster()
+      } catch {
+        case e: Exception =>
+          logWarning("Failed to connect to master", e)
+          markDisconnected()
+          context.stop(self)
+      }
+    }
+
+    def tryRegisterAllMasters() {
+      for (masterUrl <- masterUrls) {
+        logInfo("Connecting to master " + masterUrl + "...")
+        val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
+        actor ! RegisterApplication(appDescription)
+      }
+    }
+
+    def registerWithMaster() {
+      tryRegisterAllMasters()
+
+      import context.dispatcher
+      var retries = 0
+      lazy val retryTimer: Cancellable =
+        context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
+          retries += 1
+          if (registered) {
+            retryTimer.cancel()
+          } else if (retries >= REGISTRATION_RETRIES) {
+            logError("All masters are unresponsive! Giving up.")
+            markDead()
+          } else {
+            tryRegisterAllMasters()
+          }
+        }
+      retryTimer // start timer
+    }
+
+    def changeMaster(url: String) {
+      activeMasterUrl = url
+      master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
+      masterAddress = activeMasterUrl match {
+        case Master.sparkUrlRegex(host, port) =>
+          Address("akka.tcp", Master.systemName, host, port.toInt)
+        case x =>
+          throw new SparkException("Invalid spark URL: " + x)
+      }
+    }
+
+    override def receive = {
+      case RegisteredApplication(appId_, masterUrl) =>
+        appId = appId_
+        registered = true
+        changeMaster(masterUrl)
+        listener.connected(appId)
+
+      case ApplicationRemoved(message) =>
+        logError("Master removed our application: %s; stopping client".format(message))
+        markDisconnected()
+        context.stop(self)
+
+      case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) =>
+        val fullId = appId + "/" + id
+        logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores))
+        listener.executorAdded(fullId, workerId, hostPort, cores, memory)
+
+      case ExecutorUpdated(id, state, message, exitStatus) =>
+        val fullId = appId + "/" + id
+        val messageText = message.map(s => " (" + s + ")").getOrElse("")
+        logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText))
+        if (ExecutorState.isFinished(state)) {
+          listener.executorRemoved(fullId, message.getOrElse(""), exitStatus)
+        }
+
+      case MasterChanged(masterUrl, masterWebUiUrl) =>
+        logInfo("Master has changed, new master is at " + masterUrl)
+        changeMaster(masterUrl)
+        alreadyDisconnected = false
+        sender ! MasterChangeAcknowledged(appId)
+
+      case DisassociatedEvent(_, address, _) if address == masterAddress =>
+        logWarning(s"Connection to $address failed; waiting for master to reconnect...")
+        markDisconnected()
+
+      case StopClient =>
+        markDead()
+        sender ! true
+        context.stop(self)
+    }
+
+    /**
+     * Notify the listener that we disconnected, if we hadn't already done so before.
+     */
+    def markDisconnected() {
+      if (!alreadyDisconnected) {
+        listener.disconnected()
+        alreadyDisconnected = true
+      }
+    }
+
+    def markDead() {
+      if (!alreadyDead) {
+        listener.dead()
+        alreadyDead = true
+      }
+    }
+  }
+
+  def start() {
+    // Just launch an actor; it will call back into the listener.
+    actor = actorSystem.actorOf(Props(new ClientActor))
+  }
+
+  def stop() {
+    if (actor != null) {
+      try {
+        val timeout = AkkaUtils.askTimeout
+        val future = actor.ask(StopClient)(timeout)
+        Await.result(future, timeout)
+      } catch {
+        case e: TimeoutException =>
+          logInfo("Stop request to Master timed out; it may already be shut down.")
+      }
+      actor = null
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1070b566/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala
new file mode 100644
index 0000000..55d4ef1
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.client
+
+/**
+ * Callbacks invoked by deploy client when various events happen. There are currently four events:
+ * connecting to the cluster, disconnecting, being given an executor, and having an executor
+ * removed (either due to failure or due to revocation).
+ *
+ * Users of this API should *not* block inside the callback methods.
+ */
+private[spark] trait AppClientListener {
+  def connected(appId: String): Unit
+
+  /** Disconnection may be a temporary state, as we fail over to a new Master. */
+  def disconnected(): Unit
+
+  /** Dead means that we couldn't find any Masters to connect to, and have given up. */
+  def dead(): Unit
+
+  def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit
+
+  def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1070b566/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
deleted file mode 100644
index 953755e..0000000
--- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.deploy.client
-
-import java.util.concurrent.TimeoutException
-
-import scala.concurrent.duration._
-import scala.concurrent.Await
-
-import akka.actor._
-import akka.pattern.ask
-import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent}
-
-import org.apache.spark.{SparkException, Logging}
-import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
-import org.apache.spark.deploy.DeployMessages._
-import org.apache.spark.deploy.master.Master
-import org.apache.spark.util.AkkaUtils
-
-
-/**
- * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
- * and a listener for cluster events, and calls back the listener when various events occur.
- *
- * @param masterUrls Each url should look like spark://host:port.
- */
-private[spark] class Client(
-    actorSystem: ActorSystem,
-    masterUrls: Array[String],
-    appDescription: ApplicationDescription,
-    listener: ClientListener)
-  extends Logging {
-
-  val REGISTRATION_TIMEOUT = 20.seconds
-  val REGISTRATION_RETRIES = 3
-
-  var masterAddress: Address = null
-  var actor: ActorRef = null
-  var appId: String = null
-  var registered = false
-  var activeMasterUrl: String = null
-
-  class ClientActor extends Actor with Logging {
-    var master: ActorSelection = null
-    var alreadyDisconnected = false  // To avoid calling listener.disconnected() multiple times
-    var alreadyDead = false  // To avoid calling listener.dead() multiple times
-
-    override def preStart() {
-      context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
-      try {
-        registerWithMaster()
-      } catch {
-        case e: Exception =>
-          logWarning("Failed to connect to master", e)
-          markDisconnected()
-          context.stop(self)
-      }
-    }
-
-    def tryRegisterAllMasters() {
-      for (masterUrl <- masterUrls) {
-        logInfo("Connecting to master " + masterUrl + "...")
-        val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
-        actor ! RegisterApplication(appDescription)
-      }
-    }
-
-    def registerWithMaster() {
-      tryRegisterAllMasters()
-
-      import context.dispatcher
-      var retries = 0
-      lazy val retryTimer: Cancellable =
-        context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
-          retries += 1
-          if (registered) {
-            retryTimer.cancel()
-          } else if (retries >= REGISTRATION_RETRIES) {
-            logError("All masters are unresponsive! Giving up.")
-            markDead()
-          } else {
-            tryRegisterAllMasters()
-          }
-        }
-      retryTimer // start timer
-    }
-
-    def changeMaster(url: String) {
-      activeMasterUrl = url
-      master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
-      masterAddress = activeMasterUrl match {
-        case Master.sparkUrlRegex(host, port) =>
-          Address("akka.tcp", Master.systemName, host, port.toInt)
-        case x =>
-          throw new SparkException("Invalid spark URL: " + x)
-      }
-    }
-
-    override def receive = {
-      case RegisteredApplication(appId_, masterUrl) =>
-        appId = appId_
-        registered = true
-        changeMaster(masterUrl)
-        listener.connected(appId)
-
-      case ApplicationRemoved(message) =>
-        logError("Master removed our application: %s; stopping client".format(message))
-        markDisconnected()
-        context.stop(self)
-
-      case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) =>
-        val fullId = appId + "/" + id
-        logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores))
-        listener.executorAdded(fullId, workerId, hostPort, cores, memory)
-
-      case ExecutorUpdated(id, state, message, exitStatus) =>
-        val fullId = appId + "/" + id
-        val messageText = message.map(s => " (" + s + ")").getOrElse("")
-        logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText))
-        if (ExecutorState.isFinished(state)) {
-          listener.executorRemoved(fullId, message.getOrElse(""), exitStatus)
-        }
-
-      case MasterChanged(masterUrl, masterWebUiUrl) =>
-        logInfo("Master has changed, new master is at " + masterUrl)
-        changeMaster(masterUrl)
-        alreadyDisconnected = false
-        sender ! MasterChangeAcknowledged(appId)
-
-      case DisassociatedEvent(_, address, _) if address == masterAddress =>
-        logWarning(s"Connection to $address failed; waiting for master to reconnect...")
-        markDisconnected()
-
-      case StopClient =>
-        markDead()
-        sender ! true
-        context.stop(self)
-    }
-
-    /**
-     * Notify the listener that we disconnected, if we hadn't already done so before.
-     */
-    def markDisconnected() {
-      if (!alreadyDisconnected) {
-        listener.disconnected()
-        alreadyDisconnected = true
-      }
-    }
-
-    def markDead() {
-      if (!alreadyDead) {
-        listener.dead()
-        alreadyDead = true
-      }
-    }
-  }
-
-  def start() {
-    // Just launch an actor; it will call back into the listener.
-    actor = actorSystem.actorOf(Props(new ClientActor))
-  }
-
-  def stop() {
-    if (actor != null) {
-      try {
-        val timeout = AkkaUtils.askTimeout
-        val future = actor.ask(StopClient)(timeout)
-        Await.result(future, timeout)
-      } catch {
-        case e: TimeoutException =>
-          logInfo("Stop request to Master timed out; it may already be shut down.")
-      }
-      actor = null
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1070b566/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
deleted file mode 100644
index be7a11b..0000000
--- a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.deploy.client
-
-/**
- * Callbacks invoked by deploy client when various events happen. There are currently four events:
- * connecting to the cluster, disconnecting, being given an executor, and having an executor
- * removed (either due to failure or due to revocation).
- *
- * Users of this API should *not* block inside the callback methods.
- */
-private[spark] trait ClientListener {
-  def connected(appId: String): Unit
-
-  /** Disconnection may be a temporary state, as we fail over to a new Master. */
-  def disconnected(): Unit
-
-  /** Dead means that we couldn't find any Masters to connect to, and have given up. */
-  def dead(): Unit
-
-  def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit
-
-  def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1070b566/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index 5b62d3b..9359bf1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@ -23,7 +23,7 @@ import org.apache.spark.deploy.{Command, ApplicationDescription}
 
 private[spark] object TestClient {
 
-  class TestListener extends ClientListener with Logging {
+  class TestListener extends AppClientListener with Logging {
     def connected(id: String) {
       logInfo("Connected to master, got app ID " + id)
     }
@@ -49,7 +49,7 @@ private[spark] object TestClient {
     val desc = new ApplicationDescription(
       "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored")
     val listener = new TestListener
-    val client = new Client(actorSystem, Array(url), desc, listener)
+    val client = new AppClient(actorSystem, Array(url), desc, listener)
     client.start()
     actorSystem.awaitTermination()
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1070b566/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 404ce7a..4da49c0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster
 import scala.collection.mutable.HashMap
 
 import org.apache.spark.{Logging, SparkContext}
-import org.apache.spark.deploy.client.{Client, ClientListener}
+import org.apache.spark.deploy.client.{AppClient, AppClientListener}
 import org.apache.spark.deploy.{Command, ApplicationDescription}
 import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl}
 import org.apache.spark.util.Utils
@@ -31,10 +31,10 @@ private[spark] class SparkDeploySchedulerBackend(
     masters: Array[String],
     appName: String)
   extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
-  with ClientListener
+  with AppClientListener
   with Logging {
 
-  var client: Client = null
+  var client: AppClient = null
   var stopping = false
   var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _
 
@@ -54,7 +54,7 @@ private[spark] class SparkDeploySchedulerBackend(
     val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
         "http://" + sc.ui.appUIAddress)
 
-    client = new Client(sc.env.actorSystem, masters, appDesc, this)
+    client = new AppClient(sc.env.actorSystem, masters, appDesc, this)
     client.start()
   }
 


[36/37] git commit: Some usability improvements

Posted by pw...@apache.org.
Some usability improvements


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

Branch: refs/heads/master
Commit: 67b9a33628b9934804c36620d8cbc73ef70106ce
Parents: 0f9d2ac
Author: Patrick Wendell <pw...@gmail.com>
Authored: Thu Jan 9 12:42:37 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Thu Jan 9 12:42:37 2014 -0800

----------------------------------------------------------------------
 .../apache/spark/deploy/ClientArguments.scala   | 14 +++++++-
 .../spark/deploy/master/ui/IndexPage.scala      | 35 ++++++++++++------
 .../spark/deploy/worker/ui/IndexPage.scala      | 38 +++++++++++++-------
 3 files changed, 62 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/67b9a336/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
index 50b92e1..db67c6d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.deploy
 
+import java.net.URL
+
 import scala.collection.mutable.ListBuffer
 
 import org.apache.log4j.Level
@@ -68,8 +70,18 @@ private[spark] class ClientArguments(args: Array[String]) {
 
     case "launch" :: _master :: _jarUrl :: _mainClass :: tail =>
       cmd = "launch"
-      master = _master
+
+      try {
+        new URL(_jarUrl)
+      } catch {
+        case e: Exception =>
+          println(s"Jar url '${_jarUrl}' is not a valid URL.")
+          println(s"Jar must be in URL format (e.g. hdfs://XX, file://XX)")
+          printUsageAndExit(-1)
+      }
+
       jarUrl = _jarUrl
+      master = _master
       mainClass = _mainClass
       _driverOptions ++= tail
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/67b9a336/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index db1dde5..a9af8df 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -63,6 +63,10 @@ private[spark] class IndexPage(parent: MasterWebUI) {
     val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse
     val completedDriversTable = UIUtils.listingTable(driverHeaders, driverRow, completedDrivers)
 
+    // For now we only show driver information if the user has submitted drivers to the cluster.
+    // This is until we integrate the notion of drivers and applications in the UI.
+    def hasDrivers = activeDrivers.length > 0 || completedDrivers.length > 0
+
     val content =
         <div class="row-fluid">
           <div class="span12">
@@ -98,26 +102,35 @@ private[spark] class IndexPage(parent: MasterWebUI) {
           </div>
         </div>
 
-        <div class="row-fluid">
-          <div class="span12">
-            <h4> Completed Applications </h4>
-            {completedAppsTable}
+        <div>
+          {if (hasDrivers)
+          <div class="row-fluid">
+            <div class="span12">
+              <h4> Running Drivers </h4>
+              {activeDriversTable}
+            </div>
           </div>
+          }
         </div>
 
         <div class="row-fluid">
           <div class="span12">
-            <h4> Active Drivers </h4>
-            {activeDriversTable}
+            <h4> Completed Applications </h4>
+            {completedAppsTable}
           </div>
         </div>
 
-        <div class="row-fluid">
-          <div class="span12">
-            <h4> Completed Drivers </h4>
-            {completedDriversTable}
+        <div>
+          {if (hasDrivers)
+          <div class="row-fluid">
+            <div class="span12">
+              <h4> Completed Drivers </h4>
+              {completedDriversTable}
+            </div>
           </div>
+          }
         </div>;
+
     UIUtils.basicSparkPage(content, "Spark Master at " + state.uri)
   }
 
@@ -170,7 +183,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
       <td sorttable_customkey={driver.desc.mem.toString}>
         {Utils.megabytesToString(driver.desc.mem.toLong)}
       </td>
-      <td>{driver.desc.command.mainClass}</td>
+      <td>{driver.desc.command.arguments(1)}</td>
     </tr>
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/67b9a336/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index 0a7f56e..925c6fb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -58,6 +58,10 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
     val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse
     def finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers)
 
+    // For now we only show driver information if the user has submitted drivers to the cluster.
+    // This is until we integrate the notion of drivers and applications in the UI.
+    def hasDrivers = runningDrivers.length > 0 || finishedDrivers.length > 0
+
     val content =
         <div class="row-fluid"> <!-- Worker Details -->
           <div class="span12">
@@ -81,6 +85,17 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
           </div>
         </div>
 
+        <div>
+          {if (hasDrivers)
+            <div class="row-fluid"> <!-- Running Drivers -->
+              <div class="span12">
+                <h4> Running Drivers {workerState.drivers.size} </h4>
+                {runningDriverTable}
+              </div>
+            </div>
+          }
+        </div>
+
         <div class="row-fluid"> <!-- Finished Executors  -->
           <div class="span12">
             <h4> Finished Executors </h4>
@@ -88,18 +103,15 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
           </div>
         </div>
 
-        <div class="row-fluid"> <!-- Running Drivers -->
-          <div class="span12">
-            <h4> Running Drivers {workerState.drivers.size} </h4>
-            {runningDriverTable}
-          </div>
-        </div>
-
-        <div class="row-fluid"> <!-- Finished Drivers  -->
-          <div class="span12">
-            <h4> Finished Drivers </h4>
-            {finishedDriverTable}
-          </div>
+        <div>
+          {if (hasDrivers)
+            <div class="row-fluid"> <!-- Finished Drivers  -->
+              <div class="span12">
+                <h4> Finished Drivers </h4>
+                {finishedDriverTable}
+              </div>
+            </div>
+          }
         </div>;
 
     UIUtils.basicSparkPage(content, "Spark Worker at %s:%s".format(
@@ -133,7 +145,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
   def driverRow(driver: DriverRunner): Seq[Node] = {
     <tr>
       <td>{driver.driverId}</td>
-      <td>{driver.driverDesc.command.mainClass}</td>
+      <td>{driver.driverDesc.command.arguments(1)}</td>
       <td>{driver.finalState.getOrElse(DriverState.RUNNING)}</td>
       <td sorttable_customkey={driver.driverDesc.cores.toString}>
         {driver.driverDesc.cores.toString}


[30/37] git commit: Rename to Client

Posted by pw...@apache.org.
Rename to Client


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

Branch: refs/heads/master
Commit: d0533f704681adccc8fe2b814dc9e5082646057a
Parents: 3d939e5
Author: Patrick Wendell <pw...@gmail.com>
Authored: Tue Jan 7 23:38:46 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Tue Jan 7 23:38:51 2014 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/deploy/Client.scala  | 114 ++++++++++++++++++
 .../apache/spark/deploy/ClientArguments.scala   | 105 +++++++++++++++++
 .../spark/deploy/client/DriverClient.scala      | 117 -------------------
 .../deploy/client/DriverClientArguments.scala   | 105 -----------------
 4 files changed, 219 insertions(+), 222 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0533f70/core/src/main/scala/org/apache/spark/deploy/Client.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala
new file mode 100644
index 0000000..0475bb1
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala
@@ -0,0 +1,114 @@
+/*
+ * 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
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.Map
+import scala.concurrent._
+
+import akka.actor._
+import org.apache.log4j.{Level, Logger}
+
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.master.Master
+import org.apache.spark.util.{AkkaUtils, Utils}
+
+/**
+ * Actor that sends a single message to the standalone master and returns the response in the
+ * given promise.
+ */
+class DriverActor(master: String, response: Promise[(Boolean, String)]) extends Actor with Logging {
+  override def receive = {
+    case SubmitDriverResponse(success, message) => {
+      response.success((success, message))
+    }
+
+    case KillDriverResponse(success, message) => {
+      response.success((success, message))
+    }
+
+    // Relay all other messages to the master.
+    case message => {
+      logInfo(s"Sending message to master $master...")
+      val masterActor = context.actorSelection(Master.toAkkaUrl(master))
+      masterActor ! message
+    }
+  }
+}
+
+/**
+ * Executable utility for starting and terminating drivers inside of a standalone cluster.
+ */
+object DriverClient {
+
+  def main(args: Array[String]) {
+    val driverArgs = new ClientArguments(args)
+    val conf = new SparkConf()
+
+    if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) {
+      conf.set("spark.akka.logLifecycleEvents", "true")
+    }
+    conf.set("spark.akka.askTimeout", "5")
+    Logger.getRootLogger.setLevel(driverArgs.logLevel)
+
+    // TODO: See if we can initialize akka so return messages are sent back using the same TCP
+    //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
+    val (actorSystem, _) = AkkaUtils.createActorSystem(
+      "driverClient", Utils.localHostName(), 0, false, conf)
+    val master = driverArgs.master
+    val response = promise[(Boolean, String)]
+    val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response)))
+
+    println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}")
+    driverArgs.cmd match {
+      case "launch" =>
+        // TODO: We could add an env variable here and intercept it in `sc.addJar` that would
+        //       truncate filesystem paths similar to what YARN does. For now, we just require
+        //       people call `addJar` assuming the jar is in the same directory.
+        val env = Map[String, String]()
+        System.getenv().foreach{case (k, v) => env(k) = v}
+
+        val mainClass = "org.apache.spark.deploy.worker.DriverWrapper"
+        val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++
+          driverArgs.driverOptions, env)
+
+        val driverDescription = new DriverDescription(
+          driverArgs.jarUrl,
+          driverArgs.memory,
+          driverArgs.cores,
+          driverArgs.supervise,
+          command)
+        driver ! RequestSubmitDriver(driverDescription)
+
+      case "kill" =>
+        val driverId = driverArgs.driverId
+        driver ! RequestKillDriver(driverId)
+    }
+
+    val (success, message) =
+      try {
+        Await.result(response.future, AkkaUtils.askTimeout(conf))
+      } catch {
+        case e: TimeoutException => (false, s"Error: Timed out sending message to $master")
+      }
+    println(message)
+    actorSystem.shutdown()
+    actorSystem.awaitTermination()
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0533f70/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
new file mode 100644
index 0000000..50b92e1
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -0,0 +1,105 @@
+/*
+ * 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
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.log4j.Level
+
+/**
+ * Command-line parser for the driver client.
+ */
+private[spark] class ClientArguments(args: Array[String]) {
+  val defaultCores = 1
+  val defaultMemory = 512
+
+  var cmd: String = "" // 'launch' or 'kill'
+  var logLevel = Level.WARN
+
+  // launch parameters
+  var master: String = ""
+  var jarUrl: String = ""
+  var mainClass: String = ""
+  var supervise: Boolean = false
+  var memory: Int = defaultMemory
+  var cores: Int = defaultCores
+  private var _driverOptions = ListBuffer[String]()
+  def driverOptions = _driverOptions.toSeq
+
+  // kill parameters
+  var driverId: String = ""
+  
+  parse(args.toList)
+
+  def parse(args: List[String]): Unit = args match {
+    case ("--cores" | "-c") :: value :: tail =>
+      cores = value.toInt
+      parse(tail)
+
+    case ("--memory" | "-m") :: value :: tail =>
+      memory = value.toInt
+      parse(tail)
+
+    case ("--supervise" | "-s") :: tail =>
+      supervise = true
+      parse(tail)
+
+    case ("--help" | "-h") :: tail =>
+      printUsageAndExit(0)
+
+    case ("--verbose" | "-v") :: tail =>
+      logLevel = Level.INFO
+      parse(tail)
+
+    case "launch" :: _master :: _jarUrl :: _mainClass :: tail =>
+      cmd = "launch"
+      master = _master
+      jarUrl = _jarUrl
+      mainClass = _mainClass
+      _driverOptions ++= tail
+
+    case "kill" :: _master :: _driverId :: tail =>
+      cmd = "kill"
+      master = _master
+      driverId = _driverId
+
+    case _ =>
+      printUsageAndExit(1)
+  }
+
+  /**
+   * Print usage and exit JVM with the given exit code.
+   */
+  def printUsageAndExit(exitCode: Int) {
+    // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars
+    //       separately similar to in the YARN client.
+    val usage =
+      s"""
+        |Usage: DriverClient [options] launch <active-master> <jar-url> <main-class> [driver options]
+        |Usage: DriverClient kill <active-master> <driver-id>
+        |
+        |Options:
+        |   -c CORES, --cores CORES        Number of cores to request (default: $defaultCores)
+        |   -m MEMORY, --memory MEMORY     Megabytes of memory to request (default: $defaultMemory)
+        |   -s, --supervise                Whether to restart the driver on failure
+        |   -v, --verbose                  Print more debugging output
+      """.stripMargin
+    System.err.println(usage)
+    System.exit(exitCode)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0533f70/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
deleted file mode 100644
index 8b066ba..0000000
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.deploy.client
-
-import scala.collection.JavaConversions._
-import scala.collection.mutable.Map
-import scala.concurrent._
-
-import akka.actor._
-import akka.actor.Actor
-
-import org.apache.spark.{Logging, SparkConf}
-import org.apache.spark.deploy.{Command, DriverDescription}
-import org.apache.spark.deploy.DeployMessages._
-import org.apache.spark.deploy.master.Master
-import org.apache.spark.util.{AkkaUtils, Utils}
-import org.apache.log4j.{Logger, Level}
-import akka.remote.RemotingLifecycleEvent
-
-/**
- * Actor that sends a single message to the standalone master and returns the response in the
- * given promise.
- */
-class DriverActor(master: String, response: Promise[(Boolean, String)]) extends Actor with Logging {
-  override def receive = {
-    case SubmitDriverResponse(success, message) => {
-      response.success((success, message))
-    }
-
-    case KillDriverResponse(success, message) => {
-      response.success((success, message))
-    }
-
-    // Relay all other messages to the master.
-    case message => {
-      logInfo(s"Sending message to master $master...")
-      val masterActor = context.actorSelection(Master.toAkkaUrl(master))
-      masterActor ! message
-    }
-  }
-}
-
-/**
- * Executable utility for starting and terminating drivers inside of a standalone cluster.
- */
-object DriverClient {
-
-  def main(args: Array[String]) {
-    val driverArgs = new DriverClientArguments(args)
-    val conf = new SparkConf()
-
-    if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) {
-      conf.set("spark.akka.logLifecycleEvents", "true")
-    }
-    conf.set("spark.akka.askTimeout", "5")
-    Logger.getRootLogger.setLevel(driverArgs.logLevel)
-
-    // TODO: See if we can initialize akka so return messages are sent back using the same TCP
-    //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
-    val (actorSystem, _) = AkkaUtils.createActorSystem(
-      "driverClient", Utils.localHostName(), 0, false, conf)
-    val master = driverArgs.master
-    val response = promise[(Boolean, String)]
-    val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response)))
-
-    println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}")
-    driverArgs.cmd match {
-      case "launch" =>
-        // TODO: We could add an env variable here and intercept it in `sc.addJar` that would
-        //       truncate filesystem paths similar to what YARN does. For now, we just require
-        //       people call `addJar` assuming the jar is in the same directory.
-        val env = Map[String, String]()
-        System.getenv().foreach{case (k, v) => env(k) = v}
-
-        val mainClass = "org.apache.spark.deploy.worker.DriverWrapper"
-        val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++
-          driverArgs.driverOptions, env)
-
-        val driverDescription = new DriverDescription(
-          driverArgs.jarUrl,
-          driverArgs.memory,
-          driverArgs.cores,
-          driverArgs.supervise,
-          command)
-        driver ! RequestSubmitDriver(driverDescription)
-
-      case "kill" =>
-        val driverId = driverArgs.driverId
-        driver ! RequestKillDriver(driverId)
-    }
-
-    val (success, message) =
-      try {
-        Await.result(response.future, AkkaUtils.askTimeout(conf))
-      } catch {
-        case e: TimeoutException => (false, s"Error: Timed out sending message to $master")
-      }
-    println(message)
-    actorSystem.shutdown()
-    actorSystem.awaitTermination()
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0533f70/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
deleted file mode 100644
index 7774a56..0000000
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.deploy.client
-
-import scala.collection.mutable.ListBuffer
-
-import org.apache.log4j.Level
-
-/**
- * Command-line parser for the driver client.
- */
-private[spark] class DriverClientArguments(args: Array[String]) {
-  val defaultCores = 1
-  val defaultMemory = 512
-
-  var cmd: String = "" // 'launch' or 'kill'
-  var logLevel = Level.WARN
-
-  // launch parameters
-  var master: String = ""
-  var jarUrl: String = ""
-  var mainClass: String = ""
-  var supervise: Boolean = false
-  var memory: Int = defaultMemory
-  var cores: Int = defaultCores
-  private var _driverOptions = ListBuffer[String]()
-  def driverOptions = _driverOptions.toSeq
-
-  // kill parameters
-  var driverId: String = ""
-  
-  parse(args.toList)
-
-  def parse(args: List[String]): Unit = args match {
-    case ("--cores" | "-c") :: value :: tail =>
-      cores = value.toInt
-      parse(tail)
-
-    case ("--memory" | "-m") :: value :: tail =>
-      memory = value.toInt
-      parse(tail)
-
-    case ("--supervise" | "-s") :: tail =>
-      supervise = true
-      parse(tail)
-
-    case ("--help" | "-h") :: tail =>
-      printUsageAndExit(0)
-
-    case ("--verbose" | "-v") :: tail =>
-      logLevel = Level.INFO
-      parse(tail)
-
-    case "launch" :: _master :: _jarUrl :: _mainClass :: tail =>
-      cmd = "launch"
-      master = _master
-      jarUrl = _jarUrl
-      mainClass = _mainClass
-      _driverOptions ++= tail
-
-    case "kill" :: _master :: _driverId :: tail =>
-      cmd = "kill"
-      master = _master
-      driverId = _driverId
-
-    case _ =>
-      printUsageAndExit(1)
-  }
-
-  /**
-   * Print usage and exit JVM with the given exit code.
-   */
-  def printUsageAndExit(exitCode: Int) {
-    // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars
-    //       separately similar to in the YARN client.
-    val usage =
-      s"""
-        |Usage: DriverClient [options] launch <active-master> <jar-url> <main-class> [driver options]
-        |Usage: DriverClient kill <active-master> <driver-id>
-        |
-        |Options:
-        |   -c CORES, --cores CORES        Number of cores to request (default: $defaultCores)
-        |   -m MEMORY, --memory MEMORY     Megabytes of memory to request (default: $defaultMemory)
-        |   -s, --supervise                Whether to restart the driver on failure
-        |   -v, --verbose                  Print more debugging output
-      """.stripMargin
-    System.err.println(usage)
-    System.exit(exitCode)
-  }
-}


[18/37] git commit: Changes to allow fate sharing of drivers/executors and workers.

Posted by pw...@apache.org.
Changes to allow fate sharing of drivers/executors and workers.


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

Branch: refs/heads/master
Commit: 35f6dc252a8961189837e79914f305d0745a8792
Parents: c8c8b42
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sat Dec 28 11:08:37 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sun Dec 29 11:14:36 2013 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/SparkContext.scala   |  1 +
 .../org/apache/spark/deploy/DeployMessage.scala |  2 +-
 .../apache/spark/deploy/DriverDescription.scala |  7 +-
 .../spark/deploy/client/DriverClient.scala      | 18 ++++--
 .../deploy/client/DriverClientArguments.scala   | 23 +------
 .../org/apache/spark/deploy/master/Master.scala |  2 +-
 .../spark/deploy/master/ui/IndexPage.scala      |  2 +-
 .../spark/deploy/worker/CommandUtils.scala      | 63 ++++++++++++++++++
 .../spark/deploy/worker/DriverRunner.scala      | 50 ++++++++-------
 .../spark/deploy/worker/DriverWrapper.scala     | 30 +++++++++
 .../spark/deploy/worker/ExecutorRunner.scala    | 67 ++++----------------
 .../org/apache/spark/deploy/worker/Worker.scala | 13 +++-
 .../spark/deploy/worker/WorkerWatcher.scala     | 47 ++++++++++++++
 .../spark/deploy/worker/ui/IndexPage.scala      |  2 +-
 .../executor/CoarseGrainedExecutorBackend.scala | 27 +++++---
 .../cluster/SparkDeploySchedulerBackend.scala   |  2 +-
 .../apache/spark/deploy/JsonProtocolSuite.scala | 16 +++--
 17 files changed, 239 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/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 ad3337d..41f810d 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -622,6 +622,7 @@ class SparkContext(
       } else {
         val uri = new URI(path)
         key = uri.getScheme match {
+          // TODO: Have this load jars that are available on the driver
           // A JAR file which exists only on the driver node
           case null | "file" =>
             if (SparkHadoopUtil.get.isYarnMode()) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 7bfc377..34460d3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -158,7 +158,7 @@ private[deploy] object DeployMessages {
     assert (port > 0)
   }
 
-  // Actor System to Worker
+  // Liveness checks in various places
 
   case object SendHeartbeat
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
index 32ff6db..aba81ec 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
@@ -19,13 +19,10 @@ package org.apache.spark.deploy
 
 private[spark] class DriverDescription(
     val jarUrl: String,
-    val mainClass: String,
     val mem: Int,
     val cores: Int,
-    val options: Seq[String],
-    val javaOptions: Seq[String],
-    val envVars: Seq[(String, String)])
+    val command: Command)
   extends Serializable {
 
-  override def toString: String = s"DriverDescription ($mainClass)"
+  override def toString: String = s"DriverDescription (${command.mainClass})"
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index 8f19294..7e75563 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -17,12 +17,14 @@
 
 package org.apache.spark.deploy.client
 
+import scala.collection.JavaConversions._
+import scala.collection.mutable.Map
 import scala.concurrent._
 
 import akka.actor._
 
 import org.apache.spark.Logging
-import org.apache.spark.deploy.DriverDescription
+import org.apache.spark.deploy.{Command, DriverDescription}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.util.{AkkaUtils, Utils}
@@ -68,14 +70,20 @@ object DriverClient extends Logging {
 
     driverArgs.cmd match {
       case "launch" =>
+        // TODO: Could modify env here to pass a flag indicating Spark is in deploy-driver mode
+        //       then use that to load jars locally
+        val env = Map[String, String]()
+        System.getenv().foreach{case (k, v) => env(k) = v}
+
+        val mainClass = "org.apache.spark.deploy.worker.DriverWrapper"
+        val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++
+          driverArgs.driverOptions, env)
+
         val driverDescription = new DriverDescription(
           driverArgs.jarUrl,
-          driverArgs.mainClass,
           driverArgs.memory,
           driverArgs.cores,
-          driverArgs.driverOptions,
-          driverArgs.driverJavaOptions,
-          driverArgs.driverEnvVars)
+          command)
         driver ! RequestSubmitDriver(driverDescription)
 
       case "kill" =>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
index 0c84cc9..3875838 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
@@ -32,11 +32,7 @@ private[spark] class DriverClientArguments(args: Array[String]) {
   var memory: Int = 512
   var cores: Int = 1
   private var _driverOptions = ListBuffer[String]()
-  private var _driverJavaOptions = ListBuffer[String]()
-  private var _driverEnvVars = ListBuffer[(String, String)]()
   def driverOptions = _driverOptions.toSeq
-  def driverJavaOptions = _driverJavaOptions.toSeq
-  def driverEnvVars = _driverEnvVars.toSeq
 
   // kill parameters
   var driverId: String = ""
@@ -52,19 +48,6 @@ private[spark] class DriverClientArguments(args: Array[String]) {
       memory = value.toInt
       parse(tail)
 
-    case ("--java-option" | "-j") :: value :: tail =>
-      _driverJavaOptions += value
-      parse(tail)
-
-    case ("--environment-variable" | "-e") :: value :: tail =>
-      val parts = value.split("=")
-      if (parts.length != 2) {
-        println(s"Error - invalid environment variable (expecting K=V): $value")
-        printUsageAndExit(1)
-      }
-      _driverEnvVars += ((parts(0), parts(1)))
-      parse(tail)
-
     case ("--help" | "-h") :: tail =>
       printUsageAndExit(0)
 
@@ -92,7 +75,7 @@ private[spark] class DriverClientArguments(args: Array[String]) {
     //      1) Create an uber jar with your application and dependencies (excluding Spark)
     //      2) You'll need to add this jar using addJar(X) inside of your spark context
 
-    // TODO: It wouldnt be too hard to allow users to submit their app and dependency jars
+    // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars
     //       separately similar to in the YARN client.
     System.err.println(
       "usage: DriverClient [options] launch <active-master> <jar-url> <main-class> " +
@@ -100,9 +83,7 @@ private[spark] class DriverClientArguments(args: Array[String]) {
       "usage: DriverClient kill <active-master> <driver-id>\n\n" +
       "Options:\n" +
       "  -c CORES, --cores CORES                Number of cores to request \n" +
-      "  -m MEMORY, --memory MEMORY             Megabytes of memory to request\n" +
-      "  -o JAVA_OPT, --java-option JAVA_OPT    JVM option to pass to driver\n" +
-      "  -e K=V, --environment-variable K=V     Environment variable to pass to driver\n")
+      "  -m MEMORY, --memory MEMORY             Megabytes of memory to request\n")
     System.exit(exitCode)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index a0db2a2..efb9bf4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -179,7 +179,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         val msg = s"Can only accept driver submissions in ALIVE state. Current state: $state."
         sender ! SubmitDriverResponse(false, msg)
       } else {
-        logInfo("Driver submitted " + description.mainClass)
+        logInfo("Driver submitted " + description.command.mainClass)
         val driver = createDriver(description)
         persistenceEngine.addDriver(driver)
         waitingDrivers += driver

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index 951fc67..a72d76b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -170,7 +170,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
       <td sorttable_customkey={driver.desc.mem.toString}>
         {Utils.megabytesToString(driver.desc.mem.toLong)}
       </td>
-      <td>{driver.desc.mainClass}</td>
+      <td>{driver.desc.command.mainClass}</td>
     </tr>
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
new file mode 100644
index 0000000..785aecf
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
@@ -0,0 +1,63 @@
+package org.apache.spark.deploy.worker
+
+import java.io.{File, FileOutputStream, IOException, InputStream}
+import java.lang.System._
+
+import org.apache.spark.Logging
+import org.apache.spark.deploy.Command
+import org.apache.spark.util.Utils
+
+/**
+ ** Utilities for running commands with the spark classpath.
+ */
+object CommandUtils extends Logging {
+  private[spark] def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = {
+    val runner = getEnv("JAVA_HOME", command).map(_ + "/bin/java").getOrElse("java")
+
+    // SPARK-698: do not call the run.cmd script, as process.destroy()
+    // fails to kill a process tree on Windows
+    Seq(runner) ++ buildJavaOpts(command, memory, sparkHome) ++ Seq(command.mainClass) ++
+      command.arguments
+  }
+
+  private def getEnv(key: String, command: Command): Option[String] =
+    command.environment.get(key).orElse(Option(getenv(key)))
+
+  /**
+   * Attention: this must always be aligned with the environment variables in the run scripts and
+   * the way the JAVA_OPTS are assembled there.
+   */
+  def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = {
+    val libraryOpts = getEnv("SPARK_LIBRARY_PATH", command)
+      .map(p => List("-Djava.library.path=" + p))
+      .getOrElse(Nil)
+    val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil)
+    val userOpts = getEnv("SPARK_JAVA_OPTS", command).map(Utils.splitCommandString).getOrElse(Nil)
+    val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M")
+
+    // Figure out our classpath with the external compute-classpath script
+    val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh"
+    val classPath = Utils.executeAndGetOutput(
+      Seq(sparkHome + "/bin/compute-classpath" + ext),
+      extraEnvironment=command.environment)
+
+    Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts
+  }
+
+  /** Spawn a thread that will redirect a given stream to a file */
+  def redirectStream(in: InputStream, file: File) {
+    val out = new FileOutputStream(file, true)
+    // TODO: It would be nice to add a shutdown hook here that explains why the output is
+    //       terminating. Otherwise if the worker dies the executor logs will silently stop.
+    new Thread("redirect output to " + file) {
+      override def run() {
+        try {
+          Utils.copyStream(in, out, true)
+        } catch {
+          case e: IOException =>
+            logInfo("Redirection to " + file + " closed: " + e.getMessage)
+        }
+      }
+    }.start()
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 402ad53..8950fb7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -19,6 +19,8 @@ package org.apache.spark.deploy.worker
 
 import java.io._
 
+import scala.collection.mutable.Map
+
 import akka.actor.ActorRef
 import com.google.common.base.Charsets
 import com.google.common.io.Files
@@ -26,10 +28,9 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileUtil, Path}
 
 import org.apache.spark.Logging
-import org.apache.spark.deploy.DriverDescription
+import org.apache.spark.deploy.{Command, DriverDescription}
 import org.apache.spark.deploy.DeployMessages.DriverStateChanged
 import org.apache.spark.deploy.master.DriverState
-import org.apache.spark.util.Utils
 
 /**
  * Manages the execution of one driver, including automatically restarting the driver on failure.
@@ -37,8 +38,10 @@ import org.apache.spark.util.Utils
 private[spark] class DriverRunner(
     val driverId: String,
     val workDir: File,
+    val sparkHome: File,
     val driverDesc: DriverDescription,
-    val worker: ActorRef)
+    val worker: ActorRef,
+    val workerUrl: String)
   extends Logging {
 
   @volatile var process: Option[Process] = None
@@ -53,9 +56,17 @@ private[spark] class DriverRunner(
         try {
           val driverDir = createWorkingDirectory()
           val localJarFilename = downloadUserJar(driverDir)
-          val command = Seq("java") ++ driverDesc.javaOptions ++ Seq(s"-Xmx${driverDesc.mem}m") ++
-            Seq("-cp", localJarFilename) ++ Seq(driverDesc.mainClass) ++ driverDesc.options
-          runCommandWithRetry(command, driverDesc.envVars, driverDir)
+
+          // Make sure user application jar is on the classpath
+          // TODO: This could eventually exploit ability for driver to add jars
+          val env = Map(driverDesc.command.environment.toSeq: _*)
+          env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename"
+          val newCommand = Command(driverDesc.command.mainClass,
+            driverDesc.command.arguments.map(substituteVariables), env)
+
+          val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem,
+            sparkHome.getAbsolutePath)
+          runCommandWithRetry(command, env, driverDir)
         }
         catch {
           case e: Exception => exn = Some(e)
@@ -79,26 +90,17 @@ private[spark] class DriverRunner(
     }
   }
 
-  /** Spawn a thread that will redirect a given stream to a file */
-  def redirectStream(in: InputStream, file: File) {
-    val out = new FileOutputStream(file, true)
-    new Thread("redirect output to " + file) {
-      override def run() {
-        try {
-          Utils.copyStream(in, out, true)
-        } catch {
-          case e: IOException =>
-            logInfo("Redirection to " + file + " closed: " + e.getMessage)
-        }
-      }
-    }.start()
+  /** Replace variables in a command argument passed to us */
+  private def substituteVariables(argument: String): String = argument match {
+    case "{{WORKER_URL}}" => workerUrl
+    case other => other
   }
 
   /**
    * Creates the working directory for this driver.
    * Will throw an exception if there are errors preparing the directory.
    */
-  def createWorkingDirectory(): File = {
+  private def createWorkingDirectory(): File = {
     val driverDir = new File(workDir, driverId)
     if (!driverDir.exists() && !driverDir.mkdirs()) {
       throw new IOException("Failed to create directory " + driverDir)
@@ -110,7 +112,7 @@ private[spark] class DriverRunner(
    * Download the user jar into the supplied directory and return its local path.
    * Will throw an exception if there are errors downloading the jar.
    */
-  def downloadUserJar(driverDir: File): String = {
+  private def downloadUserJar(driverDir: File): String = {
 
     val jarPath = new Path(driverDesc.jarUrl)
 
@@ -136,7 +138,7 @@ private[spark] class DriverRunner(
   }
 
   /** Continue launching the supplied command until it exits zero or is killed. */
-  def runCommandWithRetry(command: Seq[String], envVars: Seq[(String, String)], baseDir: File) {
+  private def runCommandWithRetry(command: Seq[String], envVars: Map[String, String], baseDir: File) {
     // Time to wait between submission retries.
     var waitSeconds = 1
     var cleanExit = false
@@ -153,13 +155,13 @@ private[spark] class DriverRunner(
 
         // Redirect stdout and stderr to files
         val stdout = new File(baseDir, "stdout")
-        redirectStream(process.get.getInputStream, stdout)
+        CommandUtils.redirectStream(process.get.getInputStream, stdout)
 
         val stderr = new File(baseDir, "stderr")
         val header = "Launch Command: %s\n%s\n\n".format(
           command.mkString("\"", "\" \"", "\""), "=" * 40)
         Files.write(header, stderr, Charsets.UTF_8)
-        redirectStream(process.get.getErrorStream, stderr)
+        CommandUtils.redirectStream(process.get.getErrorStream, stderr)
       }
 
       val exitCode = process.get.waitFor()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
new file mode 100644
index 0000000..8c13b10
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -0,0 +1,30 @@
+package org.apache.spark.deploy.worker
+
+import akka.actor._
+
+import org.apache.spark.util.{AkkaUtils, Utils}
+
+/**
+ * Utility object for launching driver programs such that they share fate with the Worker process.
+ */
+object DriverWrapper {
+  def main(args: Array[String]) {
+    args.toList match {
+      case workerUrl :: mainClass :: extraArgs =>
+        val (actorSystem, boundPort) = AkkaUtils.createActorSystem("Driver",
+          Utils.localHostName(), 0)
+        actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher")
+
+        // Delegate to supplied main class
+        val clazz = Class.forName(args(1))
+        val mainMethod = clazz.getMethod("main", classOf[Array[String]])
+        mainMethod.invoke(null, extraArgs.toArray[String])
+
+        actorSystem.awaitTermination()
+
+      case _ =>
+        System.err.println("Usage: DriverWrapper <workerUrl> <driverMainClass> [options]")
+        System.exit(-1)
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index fff9cb6..2e61d39 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -18,17 +18,15 @@
 package org.apache.spark.deploy.worker
 
 import java.io._
-import java.lang.System.getenv
 
 import akka.actor.ActorRef
 
 import com.google.common.base.Charsets
 import com.google.common.io.Files
 
-import org.apache.spark.{Logging}
-import org.apache.spark.deploy.{ExecutorState, ApplicationDescription}
+import org.apache.spark.Logging
+import org.apache.spark.deploy.{ExecutorState, ApplicationDescription, Command}
 import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged
-import org.apache.spark.util.Utils
 
 /**
  * Manages the execution of one executor process.
@@ -44,16 +42,17 @@ private[spark] class ExecutorRunner(
     val host: String,
     val sparkHome: File,
     val workDir: File,
+    val workerUrl: String,
     var state: ExecutorState.Value)
   extends Logging {
 
   val fullId = appId + "/" + execId
   var workerThread: Thread = null
   var process: Process = null
-  var shutdownHook: Thread = null
 
-  private def getAppEnv(key: String): Option[String] =
-    appDesc.command.environment.get(key).orElse(Option(getenv(key)))
+  // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It mike
+  // make sense to remove this in the future.
+  var shutdownHook: Thread = null
 
   def start() {
     workerThread = new Thread("ExecutorRunner for " + fullId) {
@@ -92,57 +91,13 @@ private[spark] class ExecutorRunner(
 
   /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */
   def substituteVariables(argument: String): String = argument match {
+    case "{{WORKER_URL}}" => workerUrl
     case "{{EXECUTOR_ID}}" => execId.toString
     case "{{HOSTNAME}}" => host
     case "{{CORES}}" => cores.toString
     case other => other
   }
 
-  def buildCommandSeq(): Seq[String] = {
-    val command = appDesc.command
-    val runner = getAppEnv("JAVA_HOME").map(_ + "/bin/java").getOrElse("java")
-    // SPARK-698: do not call the run.cmd script, as process.destroy()
-    // fails to kill a process tree on Windows
-    Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++
-      (command.arguments ++ Seq(appId)).map(substituteVariables)
-  }
-
-  /**
-   * Attention: this must always be aligned with the environment variables in the run scripts and
-   * the way the JAVA_OPTS are assembled there.
-   */
-  def buildJavaOpts(): Seq[String] = {
-    val libraryOpts = getAppEnv("SPARK_LIBRARY_PATH")
-      .map(p => List("-Djava.library.path=" + p))
-      .getOrElse(Nil)
-    val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil)
-    val userOpts = getAppEnv("SPARK_JAVA_OPTS").map(Utils.splitCommandString).getOrElse(Nil)
-    val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M")
-
-    // Figure out our classpath with the external compute-classpath script
-    val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh"
-    val classPath = Utils.executeAndGetOutput(
-        Seq(sparkHome + "/bin/compute-classpath" + ext),
-        extraEnvironment=appDesc.command.environment)
-
-    Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts
-  }
-
-  /** Spawn a thread that will redirect a given stream to a file */
-  def redirectStream(in: InputStream, file: File) {
-    val out = new FileOutputStream(file, true)
-    new Thread("redirect output to " + file) {
-      override def run() {
-        try {
-          Utils.copyStream(in, out, true)
-        } catch {
-          case e: IOException =>
-            logInfo("Redirection to " + file + " closed: " + e.getMessage)
-        }
-      }
-    }.start()
-  }
-
   /**
    * Download and run the executor described in our ApplicationDescription
    */
@@ -155,7 +110,9 @@ private[spark] class ExecutorRunner(
       }
 
       // Launch the process
-      val command = buildCommandSeq()
+      val fullCommand = new Command(appDesc.command.mainClass,
+        appDesc.command.arguments.map(substituteVariables), appDesc.command.environment)
+      val command = CommandUtils.buildCommandSeq(fullCommand, memory, sparkHome.getAbsolutePath)
       logInfo("Launch command: " + command.mkString("\"", "\" \"", "\""))
       val builder = new ProcessBuilder(command: _*).directory(executorDir)
       val env = builder.environment()
@@ -172,11 +129,11 @@ private[spark] class ExecutorRunner(
 
       // Redirect its stdout and stderr to files
       val stdout = new File(executorDir, "stdout")
-      redirectStream(process.getInputStream, stdout)
+      CommandUtils.redirectStream(process.getInputStream, stdout)
 
       val stderr = new File(executorDir, "stderr")
       Files.write(header, stderr, Charsets.UTF_8)
-      redirectStream(process.getErrorStream, stderr)
+      CommandUtils.redirectStream(process.getErrorStream, stderr)
 
       // Wait for it to exit; this is actually a bad thing if it happens, because we expect to run
       // long-lived processes only. However, in the future, we might restart the executor a few

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 21ec881..4e23e0d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -45,6 +45,8 @@ private[spark] class Worker(
     cores: Int,
     memory: Int,
     masterUrls: Array[String],
+    actorSystemName: String,
+    actorName: String,
     workDirPath: String = null)
   extends Actor with Logging {
   import context.dispatcher
@@ -68,6 +70,7 @@ private[spark] class Worker(
   var masterAddress: Address = null
   var activeMasterUrl: String = ""
   var activeMasterWebUiUrl : String = ""
+  val akkaUrl = "akka.tcp://%s@%s:%s/user/%s".format(actorSystemName, host, port, actorName)
   @volatile var registered = false
   @volatile var connected = false
   val workerId = generateWorkerId()
@@ -190,6 +193,9 @@ private[spark] class Worker(
         map(e => new ExecutorDescription(e.appId, e.execId, e.cores, e.state))
       sender ! WorkerSchedulerStateResponse(workerId, execs.toList, drivers.keys.toSeq)
 
+    case Heartbeat =>
+      logInfo(s"Received heartbeat from driver ${sender.path}")
+
     case RegisterWorkerFailed(message) =>
       if (!registered) {
         logError("Worker registration failed: " + message)
@@ -202,7 +208,7 @@ private[spark] class Worker(
       } else {
         logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
         val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_,
-          self, workerId, host, new File(execSparkHome_), workDir, ExecutorState.RUNNING)
+          self, workerId, host, new File(execSparkHome_), workDir, akkaUrl, ExecutorState.RUNNING)
         executors(appId + "/" + execId) = manager
         manager.start()
         coresUsed += cores_
@@ -244,7 +250,7 @@ private[spark] class Worker(
 
     case LaunchDriver(driverId, driverDesc) => {
       logInfo(s"Asked to launch driver $driverId")
-      val driver = new DriverRunner(driverId, workDir, driverDesc, self)
+      val driver = new DriverRunner(driverId, workDir, sparkHome, driverDesc, self, akkaUrl)
       drivers(driverId) = driver
       driver.start()
 
@@ -322,9 +328,10 @@ private[spark] object Worker {
     : (ActorSystem, Int) = {
     // The LocalSparkCluster runs multiple local sparkWorkerX actor systems
     val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
+    val actorName = "Worker"
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
     actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
-      masterUrls, workDir), name = "Worker")
+      masterUrls, systemName, actorName, workDir), name = actorName)
     (actorSystem, boundPort)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
new file mode 100644
index 0000000..e4352f1
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
@@ -0,0 +1,47 @@
+package org.apache.spark.deploy.worker
+
+import akka.actor.{Actor, Address, AddressFromURIString}
+import akka.remote.{AssociatedEvent, AssociationErrorEvent, AssociationEvent, DisassociatedEvent, RemotingLifecycleEvent}
+
+import org.apache.spark.Logging
+import org.apache.spark.deploy.DeployMessages.SendHeartbeat
+
+/**
+ * Actor which connects to a worker process and terminates the JVM if the connection is severed.
+ * Provides fate sharing between a worker and its associated child processes.
+ */
+private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging {
+  override def preStart() {
+    context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
+
+    logInfo(s"Connecting to worker $workerUrl")
+    val worker = context.actorSelection(workerUrl)
+    worker ! SendHeartbeat // need to send a message here to initiate connection
+  }
+
+  // Lets us filter events only from the worker actor
+  private val expectedHostPort = AddressFromURIString(workerUrl).hostPort
+  private def isWorker(address: Address) = address.hostPort == expectedHostPort
+
+  override def receive = {
+    case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) =>
+      logInfo(s"Successfully connected to $workerUrl")
+
+    case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound)
+        if isWorker(remoteAddress) =>
+      // These logs may not be seen if the worker (and associated pipe) has died
+      logError(s"Could not initialize connection to worker $workerUrl. Exiting.")
+      logError(s"Error was: $cause")
+      System.exit(-1)
+
+    case DisassociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) =>
+      // This log message will never be seen
+      logError(s"Lost connection to worker actor $workerUrl. Exiting.")
+      System.exit(-1)
+
+    case e: AssociationEvent =>
+      // pass through association events relating to other remote actor systems
+
+    case e => logWarning(s"Received unexpected actor system event: $e")
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index 35a1507..93c6ad4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -133,7 +133,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
   def driverRow(driver: DriverRunner): Seq[Node] = {
     <tr>
       <td>{driver.driverId}</td>
-      <td>{driver.driverDesc.mainClass}</td>
+      <td>{driver.driverDesc.command.mainClass}</td>
       <td sorttable_customkey={driver.driverDesc.cores.toString}>
         {driver.driverDesc.cores.toString}
       </td>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/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 debbdd4..eb1199e 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -24,8 +24,9 @@ import akka.remote._
 
 import org.apache.spark.Logging
 import org.apache.spark.TaskState.TaskState
+import org.apache.spark.deploy.worker.WorkerWatcher
 import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
-import org.apache.spark.util.{Utils, AkkaUtils}
+import org.apache.spark.util.{AkkaUtils, Utils}
 
 private[spark] class CoarseGrainedExecutorBackend(
     driverUrl: String,
@@ -91,7 +92,8 @@ private[spark] class CoarseGrainedExecutorBackend(
 }
 
 private[spark] object CoarseGrainedExecutorBackend {
-  def run(driverUrl: String, executorId: String, hostname: String, cores: Int) {
+  def run(driverUrl: String, executorId: String, hostname: String, cores: Int,
+          workerUrl: Option[String]) {
     // Debug code
     Utils.checkHost(hostname)
 
@@ -105,17 +107,24 @@ private[spark] object CoarseGrainedExecutorBackend {
     actorSystem.actorOf(
       Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores),
       name = "Executor")
+    workerUrl.foreach{ url =>
+      actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher")
+    }
     actorSystem.awaitTermination()
   }
 
   def main(args: Array[String]) {
-    if (args.length < 4) {
-      //the reason we allow the last appid argument is to make it easy to kill rogue executors
-      System.err.println(
-        "Usage: CoarseGrainedExecutorBackend <driverUrl> <executorId> <hostname> <cores> " +
-        "[<appid>]")
-      System.exit(1)
+    args.length match {
+      case x if x < 4 =>
+        System.err.println(
+          // Worker url is used in spark standalone mode to enforce fate-sharing with worker
+          "Usage: CoarseGrainedExecutorBackend <driverUrl> <executorId> <hostname> " +
+          "<cores> [<workerUrl>]")
+        System.exit(1)
+      case 4 =>
+        run(args(0), args(1), args(2), args(3).toInt, None)
+      case x if x > 4 =>
+        run(args(0), args(1), args(2), args(3).toInt, Some(args(4)))
     }
-    run(args(0), args(1), args(2), args(3).toInt)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 4da49c0..921b887 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -47,7 +47,7 @@ private[spark] class SparkDeploySchedulerBackend(
     val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
       System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
       CoarseGrainedSchedulerBackend.ACTOR_NAME)
-    val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
+    val args = Seq(driverUrl, "{{WORKER_URL}}", "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
     val command = Command(
       "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
     val sparkHome = sc.getSparkHome().getOrElse(null)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/35f6dc25/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index 485f688..372c9f4 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -80,11 +80,14 @@ class JsonProtocolSuite extends FunSuite {
     new ApplicationInfo(3, "id", createAppDesc(), new Date(123456789), null, "appUriStr")
   }
 
-  def createDriverDesc() = new DriverDescription(
-    "hdfs://some-dir/some.jar", "org.apache.spark.FakeClass", 100, 3,
-    Seq("--some-config", "val", "--other-config", "val"), Seq("-Dsystem.property=someValue"),
-    Seq(("K1", "V1"), ("K2", "V2"))
+  def createDriverCommand() = new Command(
+    "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"),
+    Map(("K1", "V1"), ("K2", "V2"))
   )
+
+  def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3,
+    createDriverCommand())
+
   def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", createDriverDesc(), new Date())
 
   def createWorkerInfo(): WorkerInfo = {
@@ -92,10 +95,11 @@ class JsonProtocolSuite extends FunSuite {
   }
   def createExecutorRunner(): ExecutorRunner = {
     new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host",
-      new File("sparkHome"), new File("workDir"), ExecutorState.RUNNING)
+      new File("sparkHome"), new File("workDir"), "akka://worker", ExecutorState.RUNNING)
   }
   def createDriverRunner(): DriverRunner = {
-    new DriverRunner("driverId", new File("workDir"), createDriverDesc(), null)
+    new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), createDriverDesc(),
+      null, "akka://worker")
   }
 
   def assertValidJson(json: JValue) {


[19/37] git commit: Documentation and adding supervise option

Posted by pw...@apache.org.
Documentation and adding supervise option


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

Branch: refs/heads/master
Commit: 6ffa9bb226ac9ceec4a34f0011c35d2d9710f8f8
Parents: 35f6dc2
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sun Dec 29 11:26:56 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sun Dec 29 11:26:56 2013 -0800

----------------------------------------------------------------------
 .../apache/spark/deploy/DriverDescription.scala |  1 +
 .../spark/deploy/client/DriverClient.scala      |  3 +-
 .../deploy/client/DriverClientArguments.scala   | 12 ++++---
 .../spark/deploy/worker/DriverRunner.scala      | 15 ++++----
 docs/spark-standalone.md                        | 38 +++++++++++++++++---
 5 files changed, 51 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6ffa9bb2/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
index aba81ec..58c95dc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
@@ -21,6 +21,7 @@ private[spark] class DriverDescription(
     val jarUrl: String,
     val mem: Int,
     val cores: Int,
+    val supervise: Boolean,
     val command: Command)
   extends Serializable {
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6ffa9bb2/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index 7e75563..6257303 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -71,7 +71,7 @@ object DriverClient extends Logging {
     driverArgs.cmd match {
       case "launch" =>
         // TODO: Could modify env here to pass a flag indicating Spark is in deploy-driver mode
-        //       then use that to load jars locally
+        //       then use that to load jars locally (e.g. truncate the filesystem path)
         val env = Map[String, String]()
         System.getenv().foreach{case (k, v) => env(k) = v}
 
@@ -83,6 +83,7 @@ object DriverClient extends Logging {
           driverArgs.jarUrl,
           driverArgs.memory,
           driverArgs.cores,
+          driverArgs.supervise,
           command)
         driver ! RequestSubmitDriver(driverDescription)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6ffa9bb2/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
index 3875838..6a15422 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
@@ -29,6 +29,7 @@ private[spark] class DriverClientArguments(args: Array[String]) {
   var master: String = ""
   var jarUrl: String = ""
   var mainClass: String = ""
+  var supervise: Boolean = false
   var memory: Int = 512
   var cores: Int = 1
   private var _driverOptions = ListBuffer[String]()
@@ -48,6 +49,10 @@ private[spark] class DriverClientArguments(args: Array[String]) {
       memory = value.toInt
       parse(tail)
 
+    case ("--supervise" | "-s") :: tail =>
+      supervise = true
+      parse(tail)
+
     case ("--help" | "-h") :: tail =>
       printUsageAndExit(0)
 
@@ -71,10 +76,6 @@ private[spark] class DriverClientArguments(args: Array[String]) {
    * Print usage and exit JVM with the given exit code.
    */
   def printUsageAndExit(exitCode: Int) {
-    // TODO: Document the submission approach here. It is:
-    //      1) Create an uber jar with your application and dependencies (excluding Spark)
-    //      2) You'll need to add this jar using addJar(X) inside of your spark context
-
     // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars
     //       separately similar to in the YARN client.
     System.err.println(
@@ -83,7 +84,8 @@ private[spark] class DriverClientArguments(args: Array[String]) {
       "usage: DriverClient kill <active-master> <driver-id>\n\n" +
       "Options:\n" +
       "  -c CORES, --cores CORES                Number of cores to request \n" +
-      "  -m MEMORY, --memory MEMORY             Megabytes of memory to request\n")
+      "  -m MEMORY, --memory MEMORY             Megabytes of memory to request\n" +
+      "  -s, --supervise                        Whether to restart the driver on failure\n")
     System.exit(exitCode)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6ffa9bb2/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 8950fb7..41500bb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -66,7 +66,7 @@ private[spark] class DriverRunner(
 
           val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem,
             sparkHome.getAbsolutePath)
-          runCommandWithRetry(command, env, driverDir)
+          runCommand(command, env, driverDir, driverDesc.supervise)
         }
         catch {
           case e: Exception => exn = Some(e)
@@ -137,13 +137,14 @@ private[spark] class DriverRunner(
     localJarFilename
   }
 
-  /** Continue launching the supplied command until it exits zero or is killed. */
-  private def runCommandWithRetry(command: Seq[String], envVars: Map[String, String], baseDir: File) {
+  /** Launch the supplied command. */
+  private def runCommand(command: Seq[String], envVars: Map[String, String], baseDir: File,
+      supervise: Boolean) {
     // Time to wait between submission retries.
     var waitSeconds = 1
-    var cleanExit = false
+    var keepTrying = !killed
 
-    while (!cleanExit && !killed) {
+    while (keepTrying) {
       logInfo("Launch Command: " + command.mkString("\"", "\" \"", "\""))
       val builder = new ProcessBuilder(command: _*).directory(baseDir)
       envVars.map{ case(k,v) => builder.environment().put(k, v) }
@@ -166,8 +167,8 @@ private[spark] class DriverRunner(
 
       val exitCode = process.get.waitFor()
 
-      cleanExit = exitCode == 0
-      if (!cleanExit && !killed) {
+      keepTrying = supervise && exitCode != 0 && !killed
+      if (keepTrying) {
         waitSeconds = waitSeconds * 2 // exponential back-off
         logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.")
         (0 until waitSeconds).takeWhile(f => {Thread.sleep(1000); !killed})

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6ffa9bb2/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index b822265..59adbce 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -10,11 +10,7 @@ In addition to running on the Mesos or YARN cluster managers, Spark also provide
 
 # Installing Spark Standalone to a Cluster
 
-The easiest way to deploy Spark is by running the `./make-distribution.sh` script to create a binary distribution.
-This distribution can be deployed to any machine with the Java runtime installed; there is no need to install Scala.
-
-The recommended procedure is to deploy and start the master on one node first, get the master spark URL,
-then modify `conf/spark-env.sh` in the `dist/` directory before deploying to all the other nodes.
+To install Spark Standlone mode, you simply place a compiled version of Spark on each node on the cluster. You can obtain pre-built versions of Spark with each release or [build it yourself](index.html#building).
 
 # Starting a Cluster Manually
 
@@ -150,6 +146,38 @@ automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` vari
 
 You can also pass an option `-c <numCores>` to control the number of cores that spark-shell uses on the cluster.
 
+# Launching Applications Inside the Cluster
+
+You may also run your application entirely inside of the cluster by submitting your application driver using the submission client. The syntax for submitting applications is as follows:
+
+
+    ./spark-class org.apache.spark.deploy.client.DriverClient launch 
+       [client-options] \
+       <cluster-url> <application-jar-url> <main-class> \
+       [application-options]
+
+    cluster-url: The URL of the master node.
+    application-jar-url: Path to a bundled jar including your application and all dependencies.
+                         Accepts hdfs://, file://, and http:// paths.
+    main-class: The entry point for your application.
+
+    Client Options:
+      --memory <count> (amount of memory, in MB, allocated for your driver program)
+      --cores <count> (number of cores allocated for your driver program)
+      --supervise (whether to automatically restart your driver on application or node failure)
+
+Keep in mind that your driver program will be executed on a remote worker machine. You can control the execution environment in the following ways:
+
+ * _Environment variables_: These will be captured from the environment in which you launch the client and applied when launching the driver program.
+ * _Java options_: You can add java options by setting `SPARK_JAVA_OPTS` in the environment in which you launch the submission client.
+  * _Dependencies_: You'll still need to call `sc.addJar` inside of your driver program to add your application jar and any dependencies. If you submit a local application jar to the client (e.g one with a `file://` URL), it will be uploaded into the working directory of your driver program. Then, you can add it using `sc.addJar("jar-name.jar")`.
+
+Once you submit a driver program, it will appear in the cluster management UI at port 8080 and
+be assigned an identifier. If you'd like to prematurely terminate the program, you can do so using
+the same client:
+
+    ./spark-class org.apache.spark.deploy.client.DriverClient kill <driverId>
+
 # Resource Scheduling
 
 The standalone cluster mode currently only supports a simple FIFO scheduler across applications.


[37/37] git commit: Merge pull request #293 from pwendell/standalone-driver

Posted by pw...@apache.org.
Merge pull request #293 from pwendell/standalone-driver

SPARK-998: Support Launching Driver Inside of Standalone Mode

[NOTE: I need to bring the tests up to date with new changes, so for now they will fail]

This patch provides support for launching driver programs inside of a standalone cluster manager. It also supports monitoring and re-launching of driver programs which is useful for long running, recoverable applications such as Spark Streaming jobs. For those jobs, this patch allows a deployment mode which is resilient to the failure of any worker node, failure of a master node (provided a multi-master setup), and even failures of the applicaiton itself, provided they are recoverable on a restart. Driver information, such as the status and logs from a driver, is displayed in the UI

There are a few small TODO's here, but the code is generally feature-complete. They are:
- Bring tests up to date and add test coverage
- Restarting on failure should be optional and maybe off by default.
- See if we can re-use akka connections to facilitate clients behind a firewall

A sensible place to start for review would be to look at the `DriverClient` class which presents users the ability to launch their driver program. I've also added an example program (`DriverSubmissionTest`) that allows you to test this locally and play around with killing workers, etc. Most of the code is devoted to persisting driver state in the cluster manger, exposing it in the UI, and dealing correctly with various types of failures.

Instructions to test locally:
- `sbt/sbt assembly/assembly examples/assembly`
- start a local version of the standalone cluster manager

```
./spark-class org.apache.spark.deploy.client.DriverClient \
  -j -Dspark.test.property=something \
  -e SPARK_TEST_KEY=SOMEVALUE \
  launch spark://10.99.1.14:7077 \
  ../path-to-examples-assembly-jar \
  org.apache.spark.examples.DriverSubmissionTest 1000 some extra options --some-option-here -X 13
```
- Go in the UI and make sure it started correctly, look at the output etc
- Kill workers, the driver program, masters, etc.


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

Branch: refs/heads/master
Commit: d86a85e9cac09bd909a356de9181bd282c905e72
Parents: 26cdb5f 67b9a33
Author: Patrick Wendell <pw...@gmail.com>
Authored: Thu Jan 9 18:37:52 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Thu Jan 9 18:37:52 2014 -0800

----------------------------------------------------------------------
 core/pom.xml                                    |  10 +
 .../scala/org/apache/spark/deploy/Client.scala  | 151 ++++++++++++
 .../apache/spark/deploy/ClientArguments.scala   | 117 ++++++++++
 .../org/apache/spark/deploy/DeployMessage.scala |  52 ++++-
 .../apache/spark/deploy/DriverDescription.scala |  29 +++
 .../apache/spark/deploy/client/AppClient.scala  | 201 ++++++++++++++++
 .../spark/deploy/client/AppClientListener.scala |  39 ++++
 .../org/apache/spark/deploy/client/Client.scala | 200 ----------------
 .../spark/deploy/client/ClientListener.scala    |  39 ----
 .../apache/spark/deploy/client/TestClient.scala |   4 +-
 .../apache/spark/deploy/master/DriverInfo.scala |  36 +++
 .../spark/deploy/master/DriverState.scala       |  33 +++
 .../master/FileSystemPersistenceEngine.scala    |  17 +-
 .../org/apache/spark/deploy/master/Master.scala | 189 ++++++++++++++-
 .../spark/deploy/master/PersistenceEngine.scala |  11 +-
 .../apache/spark/deploy/master/WorkerInfo.scala |  20 +-
 .../master/ZooKeeperPersistenceEngine.scala     |  14 +-
 .../spark/deploy/master/ui/IndexPage.scala      |  56 ++++-
 .../spark/deploy/worker/CommandUtils.scala      |  63 +++++
 .../spark/deploy/worker/DriverRunner.scala      | 234 +++++++++++++++++++
 .../spark/deploy/worker/DriverWrapper.scala     |  31 +++
 .../spark/deploy/worker/ExecutorRunner.scala    |  67 ++----
 .../org/apache/spark/deploy/worker/Worker.scala |  63 ++++-
 .../spark/deploy/worker/WorkerWatcher.scala     |  55 +++++
 .../spark/deploy/worker/ui/IndexPage.scala      |  65 +++++-
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |  43 ++--
 .../executor/CoarseGrainedExecutorBackend.scala |  27 ++-
 .../cluster/SparkDeploySchedulerBackend.scala   |  10 +-
 .../apache/spark/deploy/JsonProtocolSuite.scala |  40 +++-
 .../spark/deploy/worker/DriverRunnerTest.scala  | 131 +++++++++++
 .../deploy/worker/ExecutorRunnerTest.scala      |   4 +-
 .../deploy/worker/WorkerWatcherSuite.scala      |  32 +++
 docs/spark-standalone.md                        |  38 ++-
 .../spark/examples/DriverSubmissionTest.scala   |  46 ++++
 pom.xml                                         |  17 ++
 project/SparkBuild.scala                        |   1 +
 36 files changed, 1800 insertions(+), 385 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d86a85e9/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d86a85e9/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d86a85e9/pom.xml
----------------------------------------------------------------------


[15/37] git commit: Minor fixes

Posted by pw...@apache.org.
Minor fixes


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

Branch: refs/heads/master
Commit: 5c1b4f64052e8fae0d942def4d6085a971faee4e
Parents: c23d640
Author: Patrick Wendell <pw...@gmail.com>
Authored: Thu Dec 26 14:14:49 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Thu Dec 26 14:39:39 2013 -0800

----------------------------------------------------------------------
 .../spark/deploy/client/DriverClient.scala      |  1 -
 .../org/apache/spark/deploy/master/Master.scala | 36 ++++++++++++--------
 .../spark/deploy/worker/ui/IndexPage.scala      |  8 ++---
 3 files changed, 25 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5c1b4f64/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index d2f3c09..8f19294 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -90,7 +90,6 @@ object DriverClient extends Logging {
         case e: TimeoutException => (false, s"Master $master failed to respond in time")
       }
     if (success) logInfo(message) else logError(message)
-    actorSystem.stop(driver)
     actorSystem.shutdown()
     actorSystem.awaitTermination()
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5c1b4f64/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 7f9ad8a..a0db2a2 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -37,6 +37,7 @@ import org.apache.spark.deploy.master.MasterMessages._
 import org.apache.spark.deploy.master.ui.MasterWebUI
 import org.apache.spark.metrics.MetricsSystem
 import org.apache.spark.util.{AkkaUtils, Utils}
+import org.apache.spark.deploy.master.DriverState.DriverState
 
 private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
   import context.dispatcher
@@ -268,21 +269,11 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }
 
     case DriverStateChanged(driverId, state, exception) => {
-      if (!(state == DriverState.FAILED || state == DriverState.FINISHED ||
-          state == DriverState.KILLED)) {
-        throw new Exception(s"Received unexpected state update for driver $driverId: $state")
-      }
-      drivers.find(_.id == driverId) match {
-        case Some(driver) => {
-          drivers -= driver
-          completedDrivers += driver
-          persistenceEngine.removeDriver(driver)
-          driver.state = state
-          driver.exception = exception
-          driver.worker.foreach(w => w.removeDriver(driver))
-        }
-        case None =>
-          logWarning(s"Got driver update for unknown driver $driverId")
+      state match {
+        case DriverState.FAILED | DriverState.FINISHED | DriverState.KILLED =>
+          removeDriver(driverId, state, exception)
+        case _ =>
+          throw new Exception(s"Received unexpected state update for driver $driverId: $state")
       }
     }
 
@@ -638,6 +629,21 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     worker.actor ! LaunchDriver(driver.id, driver.desc)
     driver.state = DriverState.RUNNING
   }
+
+  def removeDriver(driverId: String, finalState: DriverState, exception: Option[Exception]) {
+    drivers.find(d => d.id == driverId) match {
+      case Some(driver) =>
+        logInfo(s"Removing driver: $driverId")
+        drivers -= driver
+        completedDrivers += driver
+        persistenceEngine.removeDriver(driver)
+        driver.state = finalState
+        driver.exception = exception
+        driver.worker.foreach(w => w.removeDriver(driver))
+      case None =>
+        logWarning(s"Asked to remove unknown driver: $driverId")
+    }
+  }
 }
 
 private[spark] object Master {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5c1b4f64/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index c8cafac..35a1507 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -53,10 +53,10 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
       UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors)
 
     val driverHeaders = Seq("DriverID", "Main Class", "Cores", "Memory", "Logs")
-    val runningDriverTable =
-      UIUtils.listingTable(driverHeaders, driverRow, workerState.drivers)
-    def finishedDriverTable =
-      UIUtils.listingTable(driverHeaders, driverRow, workerState.finishedDrivers)
+    val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse
+    val runningDriverTable = UIUtils.listingTable(driverHeaders, driverRow, runningDrivers)
+    val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse
+    def finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers)
 
     val content =
         <div class="row-fluid"> <!-- Worker Details -->


[22/37] git commit: Slight change to retry logic

Posted by pw...@apache.org.
Slight change to retry logic


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

Branch: refs/heads/master
Commit: a8729770f5dc944444b9996716c2f6a26485a819
Parents: 8da1012
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sun Dec 29 11:57:57 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sun Dec 29 11:57:57 2013 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/deploy/worker/DriverRunner.scala     | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a8729770/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 51baa35..e8ae2d3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -167,12 +167,13 @@ private[spark] class DriverRunner(
 
       val exitCode = process.get.waitFor()
 
-      keepTrying = supervise && exitCode != 0 && !killed
-      if (keepTrying) {
+      if (supervise && exitCode != 0 && !killed) {
         waitSeconds = waitSeconds * 2 // exponential back-off
         logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.")
         (0 until waitSeconds).takeWhile(f => {Thread.sleep(1000); !killed})
       }
+
+      keepTrying = supervise && exitCode != 0 && !killed
     }
   }
 }


[29/37] git commit: Adding --verbose option to DriverClient

Posted by pw...@apache.org.
Adding --verbose option to DriverClient


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

Branch: refs/heads/master
Commit: 3d939e5fe8930368b46eb49659a9cca8f41c8768
Parents: e21a707
Author: Patrick Wendell <pw...@gmail.com>
Authored: Tue Jan 7 23:27:18 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Tue Jan 7 23:27:18 2014 -0800

----------------------------------------------------------------------
 .../apache/spark/deploy/client/DriverClient.scala | 18 ++++++++++++++----
 .../deploy/client/DriverClientArguments.scala     |  8 ++++++++
 2 files changed, 22 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3d939e5f/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index 1cd5d99..8b066ba 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -22,12 +22,15 @@ import scala.collection.mutable.Map
 import scala.concurrent._
 
 import akka.actor._
+import akka.actor.Actor
 
-import org.apache.spark.{SparkConf, Logging}
+import org.apache.spark.{Logging, SparkConf}
 import org.apache.spark.deploy.{Command, DriverDescription}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.util.{AkkaUtils, Utils}
+import org.apache.log4j.{Logger, Level}
+import akka.remote.RemotingLifecycleEvent
 
 /**
  * Actor that sends a single message to the standalone master and returns the response in the
@@ -55,12 +58,18 @@ class DriverActor(master: String, response: Promise[(Boolean, String)]) extends
 /**
  * Executable utility for starting and terminating drivers inside of a standalone cluster.
  */
-object DriverClient extends Logging {
+object DriverClient {
 
   def main(args: Array[String]) {
     val driverArgs = new DriverClientArguments(args)
     val conf = new SparkConf()
 
+    if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) {
+      conf.set("spark.akka.logLifecycleEvents", "true")
+    }
+    conf.set("spark.akka.askTimeout", "5")
+    Logger.getRootLogger.setLevel(driverArgs.logLevel)
+
     // TODO: See if we can initialize akka so return messages are sent back using the same TCP
     //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
     val (actorSystem, _) = AkkaUtils.createActorSystem(
@@ -69,6 +78,7 @@ object DriverClient extends Logging {
     val response = promise[(Boolean, String)]
     val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response)))
 
+    println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}")
     driverArgs.cmd match {
       case "launch" =>
         // TODO: We could add an env variable here and intercept it in `sc.addJar` that would
@@ -98,9 +108,9 @@ object DriverClient extends Logging {
       try {
         Await.result(response.future, AkkaUtils.askTimeout(conf))
       } catch {
-        case e: TimeoutException => (false, s"Master $master failed to respond in time")
+        case e: TimeoutException => (false, s"Error: Timed out sending message to $master")
       }
-    if (success) logInfo(message) else logError(message)
+    println(message)
     actorSystem.shutdown()
     actorSystem.awaitTermination()
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3d939e5f/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
index d9e1c8a..7774a56 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
@@ -19,6 +19,8 @@ package org.apache.spark.deploy.client
 
 import scala.collection.mutable.ListBuffer
 
+import org.apache.log4j.Level
+
 /**
  * Command-line parser for the driver client.
  */
@@ -27,6 +29,7 @@ private[spark] class DriverClientArguments(args: Array[String]) {
   val defaultMemory = 512
 
   var cmd: String = "" // 'launch' or 'kill'
+  var logLevel = Level.WARN
 
   // launch parameters
   var master: String = ""
@@ -59,6 +62,10 @@ private[spark] class DriverClientArguments(args: Array[String]) {
     case ("--help" | "-h") :: tail =>
       printUsageAndExit(0)
 
+    case ("--verbose" | "-v") :: tail =>
+      logLevel = Level.INFO
+      parse(tail)
+
     case "launch" :: _master :: _jarUrl :: _mainClass :: tail =>
       cmd = "launch"
       master = _master
@@ -90,6 +97,7 @@ private[spark] class DriverClientArguments(args: Array[String]) {
         |   -c CORES, --cores CORES        Number of cores to request (default: $defaultCores)
         |   -m MEMORY, --memory MEMORY     Megabytes of memory to request (default: $defaultMemory)
         |   -s, --supervise                Whether to restart the driver on failure
+        |   -v, --verbose                  Print more debugging output
       """.stripMargin
     System.err.println(usage)
     System.exit(exitCode)


[21/37] git commit: TODO clean-up

Posted by pw...@apache.org.
TODO clean-up


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

Branch: refs/heads/master
Commit: 8da1012f9bb9431123d3aae26d4bb78f8e5374bc
Parents: faefea3
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sun Dec 29 11:38:12 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sun Dec 29 11:38:12 2013 -0800

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/SparkContext.scala         | 1 -
 .../scala/org/apache/spark/deploy/client/DriverClient.scala     | 5 +++--
 .../scala/org/apache/spark/deploy/worker/DriverRunner.scala     | 2 +-
 .../scala/org/apache/spark/deploy/worker/ExecutorRunner.scala   | 2 +-
 .../scala/org/apache/spark/deploy/worker/WorkerWatcher.scala    | 2 +-
 5 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8da1012f/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 41f810d..ad3337d 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -622,7 +622,6 @@ class SparkContext(
       } else {
         val uri = new URI(path)
         key = uri.getScheme match {
-          // TODO: Have this load jars that are available on the driver
           // A JAR file which exists only on the driver node
           case null | "file" =>
             if (SparkHadoopUtil.get.isYarnMode()) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8da1012f/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index a5f5763..8a4cdf0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -70,8 +70,9 @@ object DriverClient extends Logging {
 
     driverArgs.cmd match {
       case "launch" =>
-        // TODO: Could modify env here to pass a flag indicating Spark is in deploy-driver mode
-        //       then use that to load jars locally (e.g. truncate the filesystem path)
+        // TODO: We could add an env variable here and intercept it in `sc.addJar` that would
+        //       truncate filesystem paths similar to what YARN does. For now, we just require
+        //       people call `addJar` assuming the jar is in the same directory.
         val env = Map[String, String]()
         System.getenv().foreach{case (k, v) => env(k) = v}
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8da1012f/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 41500bb..51baa35 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -58,7 +58,7 @@ private[spark] class DriverRunner(
           val localJarFilename = downloadUserJar(driverDir)
 
           // Make sure user application jar is on the classpath
-          // TODO: This could eventually exploit ability for driver to add jars
+          // TODO: If we add ability to submit multiple jars they should also be added here
           val env = Map(driverDesc.command.environment.toSeq: _*)
           env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename"
           val newCommand = Command(driverDesc.command.mainClass,

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8da1012f/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index 2e61d39..fdc9a34 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -50,7 +50,7 @@ private[spark] class ExecutorRunner(
   var workerThread: Thread = null
   var process: Process = null
 
-  // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It mike
+  // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might
   // make sense to remove this in the future.
   var shutdownHook: Thread = null
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8da1012f/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
index e4352f1..f4184bc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
@@ -19,7 +19,7 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging
     worker ! SendHeartbeat // need to send a message here to initiate connection
   }
 
-  // Lets us filter events only from the worker actor
+  // Lets us filter events only from the worker's actor system
   private val expectedHostPort = AddressFromURIString(workerUrl).hostPort
   private def isWorker(address: Address) = address.hostPort == expectedHostPort
 


[31/37] git commit: Fixes

Posted by pw...@apache.org.
Fixes


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

Branch: refs/heads/master
Commit: c78b381e91c9902a1510a2ed4ec5c898b51adfe8
Parents: d0533f7
Author: Patrick Wendell <pw...@gmail.com>
Authored: Tue Jan 7 23:56:04 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Jan 8 00:09:12 2014 -0800

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/deploy/Client.scala        | 2 +-
 .../scala/org/apache/spark/deploy/worker/ExecutorRunner.scala   | 2 +-
 docs/spark-standalone.md                                        | 5 +++--
 3 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c78b381e/core/src/main/scala/org/apache/spark/deploy/Client.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala
index 0475bb1..43b9b1c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala
@@ -55,7 +55,7 @@ class DriverActor(master: String, response: Promise[(Boolean, String)]) extends
 /**
  * Executable utility for starting and terminating drivers inside of a standalone cluster.
  */
-object DriverClient {
+object Client {
 
   def main(args: Array[String]) {
     val driverArgs = new ClientArguments(args)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c78b381e/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index a9cb998..18885d7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -100,7 +100,7 @@ private[spark] class ExecutorRunner(
 
   def getCommandSeq = {
     val command = Command(appDesc.command.mainClass,
-      appDesc.command.arguments.map(substituteVariables), appDesc.command.environment)
+      appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment)
     CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c78b381e/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 7da6474..ecd642c 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -151,19 +151,20 @@ You can also pass an option `-c <numCores>` to control the number of cores that
 You may also run your application entirely inside of the cluster by submitting your application driver using the submission client. The syntax for submitting applications is as follows:
 
 
-    ./spark-class org.apache.spark.deploy.client.DriverClient launch 
+    ./spark-class org.apache.spark.deploy.Client launch 
        [client-options] \
        <cluster-url> <application-jar-url> <main-class> \
        [application-options]
 
     cluster-url: The URL of the master node.
-    application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be visible from inside of your cluster, for instance, in an HDFS directory. 
+    application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. 
     main-class: The entry point for your application.
 
     Client Options:
       --memory <count> (amount of memory, in MB, allocated for your driver program)
       --cores <count> (number of cores allocated for your driver program)
       --supervise (whether to automatically restart your driver on application or node failure)
+      --verbose (prints increased logging output)
 
 Keep in mind that your driver program will be executed on a remote worker machine. You can control the execution environment in the following ways:
 


[03/37] git commit: Adding better option parsing

Posted by pw...@apache.org.
Adding better option parsing


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

Branch: refs/heads/master
Commit: 760823d3937822ea4a6d6f476815442711c605fa
Parents: 6a4acc4
Author: Patrick Wendell <pw...@gmail.com>
Authored: Tue Dec 24 23:20:34 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Dec 25 01:19:01 2013 -0800

----------------------------------------------------------------------
 .../org/apache/spark/deploy/DeployMessage.scala |  2 +-
 .../apache/spark/deploy/DriverDescription.scala |  6 +-
 .../spark/deploy/client/DriverClient.scala      | 35 ++++---
 .../deploy/client/DriverClientArguments.scala   | 97 ++++++++++++++++++++
 .../org/apache/spark/deploy/master/Master.scala |  3 +-
 .../spark/deploy/worker/DriverRunner.scala      | 25 +++--
 .../org/apache/spark/deploy/worker/Worker.scala |  6 +-
 .../spark/deploy/worker/ui/IndexPage.scala      |  6 +-
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |  4 +
 .../spark/examples/DriverSubmissionTest.scala   | 45 +++++++++
 10 files changed, 187 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 6743526..332c7e8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -84,7 +84,7 @@ private[deploy] object DeployMessages {
       sparkHome: String)
     extends DeployMessage
 
-  case class LaunchDriver(driverId: String, jarUrl: String, mainClass: String, mem: Int)
+  case class LaunchDriver(driverId: String, driverDesc: DriverDescription)
     extends DeployMessage
 
   case class KillDriver(driverId: String) extends DeployMessage

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
index 52f6b1b..32ff6db 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
@@ -20,7 +20,11 @@ package org.apache.spark.deploy
 private[spark] class DriverDescription(
     val jarUrl: String,
     val mainClass: String,
-    val mem: Integer) // TODO: Should this be Long?
+    val mem: Int,
+    val cores: Int,
+    val options: Seq[String],
+    val javaOptions: Seq[String],
+    val envVars: Seq[(String, String)])
   extends Serializable {
 
   override def toString: String = s"DriverDescription ($mainClass)"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index 482bafd..dd62172 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -34,7 +34,7 @@ import scala.concurrent.Await
 import akka.actor.Actor.emptyBehavior
 
 /**
- * Parent class for actors that to send a single message to the standalone master and then die.
+ * Actor that sends a single message to the standalone master and then shuts down.
  */
 private[spark] abstract class SingleMessageClient(
     actorSystem: ActorSystem, master: String, message: DeployMessage)
@@ -94,34 +94,31 @@ private[spark] class TerminationClient(actorSystem: ActorSystem, master: String,
 }
 
 /**
- * Callable utility for starting and terminating drivers inside of the standalone scheduler.
+ * Executable utility for starting and terminating drivers inside of a standalone cluster.
  */
 object DriverClient {
 
   def main(args: Array[String]) {
-    if (args.size < 3) {
-      println("usage: DriverClient launch <active-master> <jar-url> <main-class>")
-      println("usage: DriverClient kill <active-master> <driver-id>")
-      System.exit(-1)
-    }
+    val driverArgs = new DriverClientArguments(args)
 
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(
-      "driverSubmission", Utils.localHostName(), 0)
-
-    // TODO Should be configurable
-    val mem = 512
+      "driverClient", Utils.localHostName(), 0)
 
-    args(0) match {
+    driverArgs.cmd match {
       case "launch" =>
-        val master = args(1)
-        val jarUrl = args(2)
-        val mainClass = args(3)
-        val driverDescription = new DriverDescription(jarUrl, mainClass, mem)
-        val client = new SubmissionClient(actorSystem, master, driverDescription)
+        val driverDescription = new DriverDescription(
+          driverArgs.jarUrl,
+          driverArgs.mainClass,
+          driverArgs.memory,
+          driverArgs.cores,
+          driverArgs.driverOptions,
+          driverArgs.driverJavaOptions,
+          driverArgs.driverEnvVars)
+        val client = new SubmissionClient(actorSystem, driverArgs.master, driverDescription)
 
       case "kill" =>
-        val master = args(1)
-        val driverId = args(2)
+        val master = driverArgs.master
+        val driverId = driverArgs.driverId
         val client = new TerminationClient(actorSystem, master, driverId)
     }
     actorSystem.awaitTermination()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
new file mode 100644
index 0000000..618467c
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
@@ -0,0 +1,97 @@
+/*
+ * 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.client
+
+import scala.collection.mutable.ListBuffer
+
+/**
+ * Command-line parser for the driver client.
+ */
+private[spark] class DriverClientArguments(args: Array[String]) {
+  var cmd: String = "" // 'launch' or 'kill'
+
+  // launch parameters
+  var master: String = ""
+  var jarUrl: String = ""
+  var mainClass: String = ""
+  var memory: Int = 512
+  var cores: Int = 1
+  private var _driverOptions = ListBuffer[String]()
+  private var _driverJavaOptions = ListBuffer[String]()
+  private var _driverEnvVars = ListBuffer[(String, String)]()
+  def driverOptions = _driverOptions.toSeq
+  def driverJavaOptions = _driverJavaOptions.toSeq
+  def driverEnvVars = _driverEnvVars.toSeq
+
+  // kill parameters
+  var driverId: String = ""
+  
+  parse(args.toList)
+
+  def parse(args: List[String]): Unit = args match {
+    case ("--cores" | "-c") :: value :: tail =>
+      cores = value.toInt
+      parse(tail)
+
+    case ("--memory" | "-m") :: value :: tail =>
+      memory = value.toInt
+      parse(tail)
+
+    case ("--java-option" | "-j") :: value :: tail =>
+      _driverJavaOptions += value
+      parse(tail)
+
+    case ("--environment-variable" | "-e") :: value :: tail =>
+      val parts = value.split("=")
+      _driverEnvVars += ((parts(0), parts(1)))
+
+    case ("--help" | "-h") :: tail =>
+      printUsageAndExit(0)
+
+    case "launch" :: _master :: _jarUrl :: _mainClass :: tail =>
+      cmd = "launch"
+      master = _master
+      jarUrl = _jarUrl
+      mainClass = _mainClass
+      _driverOptions ++= tail
+
+    case "kill" :: _master :: _driverId :: tail =>
+      cmd = "kill"
+      master = _master
+      driverId = _driverId
+
+    case _ =>
+      printUsageAndExit(1)
+  }
+
+  /**
+   * Print usage and exit JVM with the given exit code.
+   */
+  def printUsageAndExit(exitCode: Int) {
+    System.err.println(
+      "usage: DriverClient launch [options] <active-master> <jar-url> <main-class> " +
+        "[driver options]\n" +
+      "usage: DriverClient kill <active-master> <driver-id>\n\n" +
+      "Options:\n" +
+      "  -c CORES, --cores CORES                Number of cores to request \n" +
+      "  -m MEMORY, --memory MEMORY             Megabytes of memory to request\n" +
+      "  -j JAVA_OPT, --java-option JAVA_OPT    Java option to pass to driver\n" +
+      "  -e K=V, --environment-variable K=V     Environment variable to pass to driver\n")
+    System.exit(exitCode)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 76af332..9bfacfc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -627,8 +627,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     logInfo("Launching driver " + driver.id + " on worker " + worker.id)
     worker.addDriver(driver)
     driver.worker = Some(worker)
-    worker.actor ! LaunchDriver(driver.id, driver.desc.jarUrl, driver.desc.mainClass,
-      driver.desc.mem)
+    worker.actor ! LaunchDriver(driver.id, driver.desc)
     driver.state = DriverState.RUNNING
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index fccc36b..41a089a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -29,16 +29,15 @@ import org.apache.hadoop.conf.Configuration
 import akka.actor.{ActorRef, ActorSelection}
 import org.apache.spark.deploy.DeployMessages.DriverStateChanged
 import org.apache.spark.deploy.master.DriverState
+import org.apache.spark.deploy.DriverDescription
 
 /**
  * Manages the execution of one driver process.
  */
 private[spark] class DriverRunner(
     val driverId: String,
-    val jarUrl: String,
-    val mainClass: String,
     val workDir: File,
-    val memory: Int,
+    val driverDesc: DriverDescription,
     val worker: ActorRef)
   extends Logging {
 
@@ -54,8 +53,9 @@ private[spark] class DriverRunner(
         try {
           val driverDir = createWorkingDirectory()
           val localJarFilename = downloadUserJar(driverDir)
-          val command = Seq("java", "-cp", localJarFilename, mainClass)
-          runCommandWithRetry(command, driverDir)
+          val command = Seq("java") ++ driverDesc.javaOptions ++ Seq("-cp", localJarFilename) ++
+            Seq(driverDesc.mainClass) ++ driverDesc.options
+          runCommandWithRetry(command, driverDesc.envVars, driverDir)
         }
         catch {
           case e: Exception => exn = Some(e)
@@ -110,7 +110,7 @@ private[spark] class DriverRunner(
    */
   def downloadUserJar(driverDir: File): String = {
 
-    val jarPath = new Path(jarUrl)
+    val jarPath = new Path(driverDesc.jarUrl)
 
     val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path
     val jarFileSystem = jarPath.getFileSystem(emptyConf)
@@ -134,17 +134,17 @@ private[spark] class DriverRunner(
   }
 
   /** Continue launching the supplied command until it exits zero. */
-  def runCommandWithRetry(command: Seq[String], baseDir: File) = {
-    /* Time to wait between submission retries. */
+  def runCommandWithRetry(command: Seq[String], envVars: Seq[(String, String)], baseDir: File) = {
+    // Time to wait between submission retries.
     var waitSeconds = 1
-    // TODO: We should distinguish between "immediate" exits and cases where it was running
-    //       for a long time and then exits.
     var cleanExit = false
 
     while (!cleanExit && !killed) {
       Thread.sleep(waitSeconds * 1000)
+
+      logInfo("Launch Command: " + command.mkString("\"", "\" \"", "\""))
       val builder = new ProcessBuilder(command: _*).directory(baseDir)
-      logInfo("Launch command: " + command.mkString("\"", "\" \"", "\""))
+      envVars.map{ case(k,v) => builder.environment().put(k, v) }
 
       process = Some(builder.start())
 
@@ -153,12 +153,11 @@ private[spark] class DriverRunner(
       redirectStream(process.get.getInputStream, stdout)
 
       val stderr = new File(baseDir, "stderr")
-      val header = "Driver Command: %s\n%s\n\n".format(
+      val header = "Launch Command: %s\n%s\n\n".format(
         command.mkString("\"", "\" \"", "\""), "=" * 40)
       Files.write(header, stderr, Charsets.UTF_8)
       redirectStream(process.get.getErrorStream, stderr)
 
-
       val exitCode =
         /* There is a race here I've elected to ignore for now because it's very unlikely and not
          * simple to fix. This could see `killed=false` then the main thread gets a kill request

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index a2b491a..dd6783a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -242,9 +242,9 @@ private[spark] class Worker(
         }
       }
 
-    case LaunchDriver(driverId, jarUrl, mainClass, memory) => {
+    case LaunchDriver(driverId, driverDesc) => {
       logInfo(s"Asked to launch driver $driverId")
-      val driver = new DriverRunner(driverId, jarUrl, mainClass, workDir, memory, self)
+      val driver = new DriverRunner(driverId, workDir, driverDesc, self)
       drivers(driverId) = driver
       driver.start()
 
@@ -278,7 +278,7 @@ private[spark] class Worker(
         master ! DriverStateChanged(driverId, state, exception)
       }
       val driver = drivers(driverId)
-      memoryUsed -= driver.memory
+      memoryUsed -= driver.driverDesc.mem
       coresUsed -= 1
       drivers -= driverId
       finishedDrivers(driverId) = driver

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index e233b82..2c37b71 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -137,9 +137,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
   def driverRow(driver: DriverRunner): Seq[Node] = {
     <tr>
       <td>{driver.driverId}</td>
-      <td>{driver.mainClass}</td>
-      <td sorttable_customkey={driver.memory.toString}>
-        {Utils.megabytesToString(driver.memory)}
+      <td>{driver.driverDesc.mainClass}</td>
+      <td sorttable_customkey={driver.driverDesc.mem.toString}>
+        {Utils.megabytesToString(driver.driverDesc.mem)}
       </td>
       <td>
         <a href={s"logPage?driverId=${driver.driverId}&logType=stdout"}>stdout</a>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index d128e58..2fd862c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -82,6 +82,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
         s"${workDir.getPath}/$appId/$executorId/$logType"
       case (None, None, Some(d)) =>
         s"${workDir.getPath}/$driverId/$logType"
+      case _ =>
+        throw new Exception("Request must specify either application or driver identifiers")
     }
 
     val (startByte, endByte) = getByteRange(path, offset, byteLength)
@@ -106,6 +108,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
         (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e")
       case (None, None, Some(d)) =>
         (s"${workDir.getPath}/$d/$logType", s"driverId=$d")
+      case _ =>
+        throw new Exception("Request must specify either application or driver identifiers")
     }
 
     val (startByte, endByte) = getByteRange(path, offset, byteLength)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/760823d3/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
new file mode 100644
index 0000000..9055ce7
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
@@ -0,0 +1,45 @@
+/*
+ * 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.examples
+
+import scala.collection.JavaConversions._
+
+/** Prints out environmental information, sleeps, and then exits. Made to
+  * test driver submission in the standalone scheduler. */
+object DriverSubmissionTest {
+  def main(args: Array[String]) {
+    if (args.size < 1) {
+      println("Usage: DriverSubmissionTest <seconds-to-sleep>")
+      System.exit(0)
+    }
+    val numSecondsToSleep = args(0).toInt
+
+    val env = System.getenv()
+    val properties = System.getProperties()
+
+    println("Environment variables containing SPARK_TEST:")
+    env.filter{case (k, v) => k.contains("SPARK_TEST")}.foreach(println)
+
+    println("System properties containing spark.test:")
+    properties.filter{case (k, v) => k.toString.contains("spark.test")}.foreach(println)
+
+    for (i <- 1 until numSecondsToSleep) {
+      Thread.sleep(1000)
+    }
+  }
+}


[25/37] git commit: Merge remote-tracking branch 'apache-github/master' into standalone-driver

Posted by pw...@apache.org.
Merge remote-tracking branch 'apache-github/master' into standalone-driver

Conflicts:
	core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
	core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
	core/src/main/scala/org/apache/spark/deploy/master/Master.scala
	core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.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/c0498f92
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/c0498f92
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/c0498f92

Branch: refs/heads/master
Commit: c0498f9265e32ba82bcf48bf0df0f29c6cfea587
Parents: f236ddd e4d6057
Author: Patrick Wendell <pw...@gmail.com>
Authored: Mon Jan 6 17:29:21 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Mon Jan 6 17:29:21 2014 -0800

----------------------------------------------------------------------
 .gitignore                                      |    2 +
 README.md                                       |   28 +-
 assembly/lib/PY4J_LICENSE.txt                   |   27 -
 assembly/lib/PY4J_VERSION.txt                   |    1 -
 assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar  |  Bin 103286 -> 0 bytes
 assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom  |    9 -
 .../net/sf/py4j/py4j/maven-metadata-local.xml   |   12 -
 assembly/pom.xml                                |   14 +-
 assembly/src/main/assembly/assembly.xml         |   11 +-
 bin/compute-classpath.cmd                       |    2 +-
 bin/compute-classpath.sh                        |    2 +-
 bin/pyspark                                     |   70 ++
 bin/pyspark.cmd                                 |   23 +
 bin/pyspark2.cmd                                |   55 +
 bin/run-example                                 |   91 ++
 bin/run-example.cmd                             |   23 +
 bin/run-example2.cmd                            |   61 ++
 bin/slaves.sh                                   |   91 --
 bin/spark-class                                 |  154 +++
 bin/spark-class.cmd                             |   23 +
 bin/spark-class2.cmd                            |   85 ++
 bin/spark-config.sh                             |   36 -
 bin/spark-daemon.sh                             |  183 ----
 bin/spark-daemons.sh                            |   35 -
 bin/spark-shell                                 |  102 ++
 bin/spark-shell.cmd                             |   23 +
 bin/start-all.sh                                |   34 -
 bin/start-master.sh                             |   52 -
 bin/start-slave.sh                              |   35 -
 bin/start-slaves.sh                             |   48 -
 bin/stop-all.sh                                 |   32 -
 bin/stop-master.sh                              |   27 -
 bin/stop-slaves.sh                              |   35 -
 core/pom.xml                                    |  422 ++++----
 .../apache/spark/network/netty/FileClient.java  |   32 +-
 .../netty/FileClientChannelInitializer.java     |    6 +-
 .../spark/network/netty/FileClientHandler.java  |   12 +-
 .../apache/spark/network/netty/FileServer.java  |   29 +-
 .../netty/FileServerChannelInitializer.java     |    3 +-
 .../spark/network/netty/FileServerHandler.java  |   18 +-
 .../org/apache/spark/default-log4j.properties   |    8 +
 .../scala/org/apache/spark/Accumulators.scala   |    8 +-
 .../scala/org/apache/spark/HttpServer.scala     |    1 +
 .../main/scala/org/apache/spark/Logging.scala   |   41 +-
 .../org/apache/spark/MapOutputTracker.scala     |   11 +-
 .../scala/org/apache/spark/Partitioner.scala    |    4 +-
 .../main/scala/org/apache/spark/SparkConf.scala |  190 ++++
 .../scala/org/apache/spark/SparkContext.scala   |  297 ++++--
 .../main/scala/org/apache/spark/SparkEnv.scala  |   54 +-
 .../scala/org/apache/spark/TaskEndReason.scala  |    2 -
 .../org/apache/spark/api/java/JavaPairRDD.scala |   36 +
 .../org/apache/spark/api/java/JavaRDDLike.scala |   11 +
 .../spark/api/java/JavaSparkContext.scala       |   56 +-
 .../org/apache/spark/api/python/PythonRDD.scala |    4 +-
 .../org/apache/spark/broadcast/Broadcast.scala  |    8 +-
 .../spark/broadcast/BroadcastFactory.scala      |    4 +-
 .../apache/spark/broadcast/HttpBroadcast.scala  |   43 +-
 .../spark/broadcast/TorrentBroadcast.scala      |   45 +-
 .../spark/deploy/FaultToleranceTest.scala       |    4 +-
 .../apache/spark/deploy/LocalSparkCluster.scala |    7 +-
 .../apache/spark/deploy/SparkHadoopUtil.scala   |   14 +-
 .../apache/spark/deploy/client/AppClient.scala  |   21 +-
 .../apache/spark/deploy/client/TestClient.scala |   10 +-
 .../org/apache/spark/deploy/master/Master.scala |   42 +-
 .../spark/deploy/master/MasterArguments.scala   |   11 +-
 .../deploy/master/SparkZooKeeperSession.scala   |    7 +-
 .../master/ZooKeeperLeaderElectionAgent.scala   |    9 +-
 .../master/ZooKeeperPersistenceEngine.scala     |    8 +-
 .../spark/deploy/master/ui/MasterWebUI.scala    |    2 +-
 .../org/apache/spark/deploy/worker/Worker.scala |   21 +-
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |   12 +-
 .../executor/CoarseGrainedExecutorBackend.scala |    6 +-
 .../org/apache/spark/executor/Executor.scala    |   47 +-
 .../org/apache/spark/io/CompressionCodec.scala  |   19 +-
 .../apache/spark/metrics/MetricsConfig.scala    |    1 -
 .../apache/spark/metrics/MetricsSystem.scala    |   11 +-
 .../spark/network/ConnectionManager.scala       |   22 +-
 .../org/apache/spark/network/ReceiverTest.scala |   12 +-
 .../org/apache/spark/network/SenderTest.scala   |   16 +-
 .../spark/network/netty/ShuffleCopier.scala     |   10 +-
 .../org/apache/spark/rdd/CheckpointRDD.scala    |   37 +-
 .../org/apache/spark/rdd/CoGroupedRDD.scala     |    2 +-
 .../org/apache/spark/rdd/PairRDDFunctions.scala |   42 +
 .../spark/rdd/PartitionerAwareUnionRDD.scala    |  110 ++
 .../main/scala/org/apache/spark/rdd/RDD.scala   |   19 +-
 .../apache/spark/rdd/RDDCheckpointData.scala    |   17 +-
 .../org/apache/spark/rdd/ShuffledRDD.scala      |    2 +-
 .../org/apache/spark/rdd/SubtractedRDD.scala    |    2 +-
 .../apache/spark/scheduler/DAGScheduler.scala   |   47 +-
 .../spark/scheduler/InputFormatInfo.scala       |   14 +-
 .../org/apache/spark/scheduler/JobLogger.scala  |    6 +-
 .../scala/org/apache/spark/scheduler/Pool.scala |    4 -
 .../org/apache/spark/scheduler/ResultTask.scala |    4 +-
 .../apache/spark/scheduler/Schedulable.scala    |    1 -
 .../spark/scheduler/SchedulableBuilder.scala    |    6 +-
 .../spark/scheduler/SchedulerBackend.scala      |    3 -
 .../apache/spark/scheduler/ShuffleMapTask.scala |    6 +-
 .../apache/spark/scheduler/SparkListener.scala  |   20 +-
 .../spark/scheduler/SparkListenerBus.scala      |    2 +-
 .../spark/scheduler/TaskResultGetter.scala      |    3 +-
 .../spark/scheduler/TaskSchedulerImpl.scala     |   32 +-
 .../apache/spark/scheduler/TaskSetManager.scala |   33 +-
 .../cluster/CoarseGrainedSchedulerBackend.scala |   20 +-
 .../cluster/SimrSchedulerBackend.scala          |    4 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |    8 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |   18 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |   12 +-
 .../spark/scheduler/local/LocalBackend.scala    |    3 +-
 .../spark/serializer/JavaSerializer.scala       |    3 +-
 .../spark/serializer/KryoSerializer.scala       |   14 +-
 .../apache/spark/serializer/Serializer.scala    |    3 +
 .../spark/serializer/SerializerManager.scala    |   23 +-
 .../spark/storage/BlockFetcherIterator.scala    |    4 +-
 .../org/apache/spark/storage/BlockManager.scala |   58 +-
 .../spark/storage/BlockManagerMaster.scala      |   11 +-
 .../spark/storage/BlockManagerMasterActor.scala |   16 +-
 .../spark/storage/BlockManagerWorker.scala      |    3 -
 .../spark/storage/BlockMessageArray.scala       |    2 -
 .../spark/storage/BlockObjectWriter.scala       |    5 +-
 .../apache/spark/storage/DiskBlockManager.scala |    2 +-
 .../spark/storage/ShuffleBlockManager.scala     |   10 +-
 .../spark/storage/StoragePerfTester.scala       |    2 +-
 .../apache/spark/storage/ThreadingTest.scala    |    8 +-
 .../scala/org/apache/spark/ui/SparkUI.scala     |    4 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |   21 +-
 .../org/apache/spark/ui/env/EnvironmentUI.scala |   15 +-
 .../spark/ui/jobs/JobProgressListener.scala     |   15 +-
 .../scala/org/apache/spark/util/AkkaUtils.scala |   39 +-
 .../org/apache/spark/util/MetadataCleaner.scala |   35 +-
 .../spark/util/SerializableHyperLogLog.scala    |   50 +
 .../org/apache/spark/util/SizeEstimator.scala   |   14 +-
 .../scala/org/apache/spark/util/Utils.scala     |   25 +-
 core/src/test/resources/spark.conf              |    8 +
 .../test/resources/uncommons-maths-1.2.2.jar    |  Bin 49019 -> 0 bytes
 .../org/apache/spark/CheckpointSuite.scala      |  363 ++++---
 .../scala/org/apache/spark/DriverSuite.scala    |    8 +-
 .../org/apache/spark/FileServerSuite.scala      |  108 +-
 .../scala/org/apache/spark/JavaAPISuite.java    |   36 +-
 .../apache/spark/MapOutputTrackerSuite.scala    |   16 +-
 .../org/apache/spark/SharedSparkContext.scala   |    4 +-
 .../scala/org/apache/spark/SparkConfSuite.scala |  110 ++
 .../deploy/worker/ExecutorRunnerTest.scala      |    4 +-
 .../apache/spark/io/CompressionCodecSuite.scala |    8 +-
 .../spark/metrics/MetricsSystemSuite.scala      |    8 +-
 .../spark/rdd/PairRDDFunctionsSuite.scala       |   34 +
 .../scala/org/apache/spark/rdd/RDDSuite.scala   |   40 +
 .../spark/scheduler/ClusterSchedulerSuite.scala |    2 +-
 .../spark/scheduler/DAGSchedulerSuite.scala     |   23 +-
 .../apache/spark/scheduler/JobLoggerSuite.scala |    4 +-
 .../spark/scheduler/SparkListenerSuite.scala    |    2 +-
 .../spark/scheduler/TaskResultGetterSuite.scala |    6 +-
 .../spark/scheduler/TaskSetManagerSuite.scala   |    4 +-
 .../spark/serializer/KryoSerializerSuite.scala  |   33 +-
 .../spark/storage/BlockManagerSuite.scala       |   97 +-
 .../spark/storage/DiskBlockManagerSuite.scala   |   18 +-
 .../apache/spark/util/SizeEstimatorSuite.scala  |    2 +-
 data/kmeans_data.txt                            |    6 +
 data/lr_data.txt                                | 1000 ++++++++++++++++++
 data/pagerank_data.txt                          |    6 +
 docs/README.md                                  |    4 +-
 docs/_config.yml                                |    2 +-
 docs/_plugins/copy_api_dirs.rb                  |    4 +-
 docs/api.md                                     |    2 +-
 docs/bagel-programming-guide.md                 |    4 +-
 docs/building-with-maven.md                     |   14 +-
 docs/configuration.md                           |   74 +-
 docs/css/bootstrap.min.css                      |    2 +-
 docs/hadoop-third-party-distributions.md        |    2 +-
 docs/index.md                                   |   16 +-
 docs/java-programming-guide.md                  |    4 +-
 docs/job-scheduling.md                          |   21 +-
 docs/mllib-guide.md                             |    2 +-
 docs/monitoring.md                              |    3 +-
 docs/python-programming-guide.md                |   45 +-
 docs/quick-start.md                             |   62 +-
 docs/running-on-mesos.md                        |   19 +-
 docs/running-on-yarn.md                         |   17 +-
 docs/scala-programming-guide.md                 |   20 +-
 docs/spark-debugger.md                          |    2 +-
 docs/spark-standalone.md                        |   35 +-
 docs/streaming-programming-guide.md             |    8 +-
 docs/tuning.md                                  |   21 +-
 ec2/spark_ec2.py                                |    2 +-
 .../org/apache/spark/examples/JavaHdfsLR.java   |    2 +-
 .../org/apache/spark/examples/JavaKMeans.java   |    2 +-
 .../org/apache/spark/examples/JavaLogQuery.java |    2 +-
 .../org/apache/spark/examples/JavaPageRank.java |    3 +-
 .../org/apache/spark/examples/JavaSparkPi.java  |    2 +-
 .../java/org/apache/spark/examples/JavaTC.java  |    2 +-
 .../apache/spark/examples/JavaWordCount.java    |    2 +-
 .../apache/spark/mllib/examples/JavaALS.java    |    2 +-
 .../apache/spark/mllib/examples/JavaKMeans.java |    2 +-
 .../org/apache/spark/mllib/examples/JavaLR.java |    2 +-
 .../streaming/examples/JavaFlumeEventCount.java |    3 +-
 .../streaming/examples/JavaKafkaWordCount.java  |    7 +-
 .../examples/JavaNetworkWordCount.java          |    3 +-
 .../streaming/examples/JavaQueueStream.java     |    2 +-
 .../apache/spark/examples/BroadcastTest.scala   |    2 +-
 .../spark/examples/ExceptionHandlingTest.scala  |    2 +-
 .../org/apache/spark/examples/GroupByTest.scala |    2 +-
 .../org/apache/spark/examples/HBaseTest.scala   |    2 +-
 .../org/apache/spark/examples/HdfsTest.scala    |    2 +-
 .../org/apache/spark/examples/LogQuery.scala    |    2 +-
 .../spark/examples/MultiBroadcastTest.scala     |    2 +-
 .../examples/SimpleSkewedGroupByTest.scala      |    2 +-
 .../spark/examples/SkewedGroupByTest.scala      |    2 +-
 .../org/apache/spark/examples/SparkALS.scala    |    2 +-
 .../org/apache/spark/examples/SparkHdfsLR.scala |    2 +-
 .../org/apache/spark/examples/SparkKMeans.scala |    2 +-
 .../org/apache/spark/examples/SparkLR.scala     |    2 +-
 .../apache/spark/examples/SparkPageRank.scala   |    2 +-
 .../org/apache/spark/examples/SparkPi.scala     |    2 +-
 .../org/apache/spark/examples/SparkTC.scala     |    2 +-
 .../examples/bagel/WikipediaPageRank.scala      |   10 +-
 .../bagel/WikipediaPageRankStandalone.scala     |    8 +-
 .../streaming/examples/ActorWordCount.scala     |    9 +-
 .../streaming/examples/FlumeEventCount.scala    |    2 +-
 .../streaming/examples/HdfsWordCount.scala      |    4 +-
 .../streaming/examples/KafkaWordCount.scala     |    4 +-
 .../streaming/examples/MQTTWordCount.scala      |    6 +-
 .../streaming/examples/NetworkWordCount.scala   |    4 +-
 .../spark/streaming/examples/QueueStream.scala  |    2 +-
 .../streaming/examples/RawNetworkGrep.scala     |    2 +-
 .../examples/StatefulNetworkWordCount.scala     |    4 +-
 .../streaming/examples/TwitterAlgebirdCMS.scala |    2 +-
 .../streaming/examples/TwitterAlgebirdHLL.scala |    2 +-
 .../streaming/examples/TwitterPopularTags.scala |    2 +-
 .../streaming/examples/ZeroMQWordCount.scala    |    6 +-
 .../clickstream/PageViewGenerator.scala         |    4 +-
 .../examples/clickstream/PageViewStream.scala   |    6 +-
 kmeans_data.txt                                 |    6 -
 lr_data.txt                                     | 1000 ------------------
 make-distribution.sh                            |   24 +-
 .../spark/mllib/api/python/PythonMLLibAPI.scala |  232 ++++
 .../spark/mllib/classification/NaiveBayes.scala |  119 +++
 .../apache/spark/mllib/recommendation/ALS.scala |   13 +-
 .../mllib/classification/NaiveBayesSuite.scala  |  108 ++
 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     |  149 ---
 .../yarn/ClientDistributedCacheManager.scala    |  228 ----
 .../spark/deploy/yarn/WorkerLauncher.scala      |  222 ----
 .../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 ----
 pagerank_data.txt                               |    6 -
 pom.xml                                         |   69 +-
 project/SparkBuild.scala                        |   52 +-
 pyspark                                         |   70 --
 pyspark.cmd                                     |   23 -
 pyspark2.cmd                                    |   55 -
 python/epydoc.conf                              |    2 +-
 python/lib/py4j-0.8.1-src.zip                   |  Bin 0 -> 37662 bytes
 python/lib/py4j0.7.egg                          |  Bin 191756 -> 0 bytes
 python/pyspark/__init__.py                      |   34 +-
 python/pyspark/broadcast.py                     |   11 +
 python/pyspark/conf.py                          |  171 +++
 python/pyspark/context.py                       |   68 +-
 python/pyspark/java_gateway.py                  |    4 +-
 python/pyspark/mllib/__init__.py                |   20 +
 python/pyspark/mllib/_common.py                 |  227 ++++
 python/pyspark/mllib/classification.py          |   86 ++
 python/pyspark/mllib/clustering.py              |   79 ++
 python/pyspark/mllib/recommendation.py          |   74 ++
 python/pyspark/mllib/regression.py              |  110 ++
 python/pyspark/rdd.py                           |   66 +-
 python/pyspark/serializers.py                   |    2 +-
 python/pyspark/shell.py                         |    4 +-
 python/pyspark/tests.py                         |    4 +-
 python/run-tests                                |    3 +-
 repl-bin/src/deb/bin/run                        |    3 +-
 repl/pom.xml                                    |    1 -
 .../org/apache/spark/repl/SparkILoop.scala      |   19 +-
 .../org/apache/spark/repl/SparkIMain.scala      |    7 +-
 run-example                                     |   91 --
 run-example.cmd                                 |   23 -
 run-example2.cmd                                |   61 --
 sbin/slaves.sh                                  |   91 ++
 sbin/spark-config.sh                            |   36 +
 sbin/spark-daemon.sh                            |  183 ++++
 sbin/spark-daemons.sh                           |   35 +
 sbin/spark-executor                             |   23 +
 sbin/start-all.sh                               |   34 +
 sbin/start-master.sh                            |   52 +
 sbin/start-slave.sh                             |   35 +
 sbin/start-slaves.sh                            |   48 +
 sbin/stop-all.sh                                |   32 +
 sbin/stop-master.sh                             |   27 +
 sbin/stop-slaves.sh                             |   35 +
 sbt/sbt                                         |   43 -
 sbt/sbt-launch-0.11.3-2.jar                     |  Bin 1096763 -> 0 bytes
 sbt/sbt.cmd                                     |   25 -
 spark-class                                     |  149 ---
 spark-class.cmd                                 |   23 -
 spark-class2.cmd                                |   85 --
 spark-executor                                  |   22 -
 spark-shell                                     |  102 --
 spark-shell.cmd                                 |   22 -
 .../org/apache/spark/streaming/Checkpoint.scala |   66 +-
 .../org/apache/spark/streaming/DStream.scala    |    4 +-
 .../apache/spark/streaming/DStreamGraph.scala   |    1 -
 .../spark/streaming/PairDStreamFunctions.scala  |   13 +-
 .../spark/streaming/StreamingContext.scala      |   95 +-
 .../streaming/api/java/JavaPairDStream.scala    |   18 +-
 .../api/java/JavaStreamingContext.scala         |   45 +-
 .../streaming/dstream/FileInputDStream.scala    |  153 +--
 .../streaming/dstream/NetworkInputDStream.scala |    8 +-
 .../streaming/dstream/ShuffledDStream.scala     |    9 +-
 .../streaming/dstream/WindowedDStream.scala     |   16 +-
 .../streaming/scheduler/JobGenerator.scala      |   71 +-
 .../streaming/scheduler/JobScheduler.scala      |    6 +-
 .../streaming/util/MasterFailureTest.scala      |    3 -
 .../spark/streaming/util/RawTextSender.scala    |    4 +-
 .../apache/spark/streaming/JavaAPISuite.java    |   10 +-
 .../spark/streaming/BasicOperationsSuite.scala  |    8 +-
 .../spark/streaming/CheckpointSuite.scala       |   59 +-
 .../spark/streaming/InputStreamsSuite.scala     |   20 +-
 .../apache/spark/streaming/TestSuiteBase.scala  |   34 +-
 .../spark/streaming/WindowOperationsSuite.scala |    5 +-
 yarn/README.md                                  |   12 +
 yarn/alpha/pom.xml                              |   32 +
 .../spark/deploy/yarn/ApplicationMaster.scala   |  464 ++++++++
 .../org/apache/spark/deploy/yarn/Client.scala   |  509 +++++++++
 .../spark/deploy/yarn/WorkerLauncher.scala      |  250 +++++
 .../spark/deploy/yarn/WorkerRunnable.scala      |  236 +++++
 .../deploy/yarn/YarnAllocationHandler.scala     |  680 ++++++++++++
 .../yarn/ApplicationMasterArguments.scala       |   94 ++
 .../spark/deploy/yarn/ClientArguments.scala     |  150 +++
 .../yarn/ClientDistributedCacheManager.scala    |  228 ++++
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala |   43 +
 .../cluster/YarnClientClusterScheduler.scala    |   48 +
 .../cluster/YarnClientSchedulerBackend.scala    |  110 ++
 .../cluster/YarnClusterScheduler.scala          |   56 +
 .../ClientDistributedCacheManagerSuite.scala    |  220 ++++
 yarn/pom.xml                                    |   84 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |  477 ---------
 .../yarn/ApplicationMasterArguments.scala       |   94 --
 .../org/apache/spark/deploy/yarn/Client.scala   |  503 ---------
 .../spark/deploy/yarn/ClientArguments.scala     |  146 ---
 .../yarn/ClientDistributedCacheManager.scala    |  228 ----
 .../spark/deploy/yarn/WorkerLauncher.scala      |  243 -----
 .../spark/deploy/yarn/WorkerRunnable.scala      |  235 ----
 .../deploy/yarn/YarnAllocationHandler.scala     |  673 ------------
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala |   43 -
 .../cluster/YarnClientClusterScheduler.scala    |   47 -
 .../cluster/YarnClientSchedulerBackend.scala    |  109 --
 .../cluster/YarnClusterScheduler.scala          |   59 --
 .../ClientDistributedCacheManagerSuite.scala    |  220 ----
 yarn/stable/pom.xml                             |   32 +
 .../spark/deploy/yarn/ApplicationMaster.scala   |  432 ++++++++
 .../org/apache/spark/deploy/yarn/Client.scala   |  525 +++++++++
 .../spark/deploy/yarn/WorkerLauncher.scala      |  230 ++++
 .../spark/deploy/yarn/WorkerRunnable.scala      |  210 ++++
 .../deploy/yarn/YarnAllocationHandler.scala     |  695 ++++++++++++
 360 files changed, 12080 insertions(+), 10645 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0498f92/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
index 4779c75,0000000..1415e2f
mode 100644,000000..100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
@@@ -1,192 -1,0 +1,201 @@@
 +/*
 + * 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.client
 +
 +import java.util.concurrent.TimeoutException
 +
- import scala.concurrent.duration._
 +import scala.concurrent.Await
++import scala.concurrent.duration._
 +
 +import akka.actor._
 +import akka.pattern.ask
- import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent}
++import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent}
 +
- import org.apache.spark.{SparkException, Logging}
++import org.apache.spark.{Logging, SparkConf, SparkException}
 +import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
 +import org.apache.spark.deploy.DeployMessages._
 +import org.apache.spark.deploy.master.Master
 +import org.apache.spark.util.AkkaUtils
 +
- 
 +/**
 + * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
 + * an app description, and a listener for cluster events, and calls back the listener when various
 + * events occur.
 + *
 + * @param masterUrls Each url should look like spark://host:port.
 + */
 +private[spark] class AppClient(
 +    actorSystem: ActorSystem,
 +    masterUrls: Array[String],
 +    appDescription: ApplicationDescription,
-     listener: AppClientListener)
++    listener: AppClientListener,
++    conf: SparkConf)
 +  extends Logging {
 +
 +  val REGISTRATION_TIMEOUT = 20.seconds
 +  val REGISTRATION_RETRIES = 3
 +
 +  var masterAddress: Address = null
 +  var actor: ActorRef = null
 +  var appId: String = null
 +  var registered = false
 +  var activeMasterUrl: String = null
 +
 +  class ClientActor extends Actor with Logging {
 +    var master: ActorSelection = null
 +    var alreadyDisconnected = false  // To avoid calling listener.disconnected() multiple times
 +    var alreadyDead = false  // To avoid calling listener.dead() multiple times
 +
 +    override def preStart() {
 +      context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
 +      try {
 +        registerWithMaster()
 +      } catch {
 +        case e: Exception =>
 +          logWarning("Failed to connect to master", e)
 +          markDisconnected()
 +          context.stop(self)
 +      }
 +    }
 +
 +    def tryRegisterAllMasters() {
 +      for (masterUrl <- masterUrls) {
 +        logInfo("Connecting to master " + masterUrl + "...")
 +        val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
 +        actor ! RegisterApplication(appDescription)
 +      }
 +    }
 +
 +    def registerWithMaster() {
 +      tryRegisterAllMasters()
 +
 +      import context.dispatcher
 +      var retries = 0
 +      lazy val retryTimer: Cancellable =
 +        context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
 +          retries += 1
 +          if (registered) {
 +            retryTimer.cancel()
 +          } else if (retries >= REGISTRATION_RETRIES) {
 +            logError("All masters are unresponsive! Giving up.")
 +            markDead()
 +          } else {
 +            tryRegisterAllMasters()
 +          }
 +        }
 +      retryTimer // start timer
 +    }
 +
 +    def changeMaster(url: String) {
 +      activeMasterUrl = url
 +      master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
 +      masterAddress = activeMasterUrl match {
 +        case Master.sparkUrlRegex(host, port) =>
 +          Address("akka.tcp", Master.systemName, host, port.toInt)
 +        case x =>
 +          throw new SparkException("Invalid spark URL: " + x)
 +      }
 +    }
 +
++    private def isPossibleMaster(remoteUrl: Address) = {
++      masterUrls.map(s => Master.toAkkaUrl(s))
++        .map(u => AddressFromURIString(u).hostPort)
++        .contains(remoteUrl.hostPort)
++    }
++
 +    override def receive = {
 +      case RegisteredApplication(appId_, masterUrl) =>
 +        appId = appId_
 +        registered = true
 +        changeMaster(masterUrl)
 +        listener.connected(appId)
 +
 +      case ApplicationRemoved(message) =>
 +        logError("Master removed our application: %s; stopping client".format(message))
 +        markDisconnected()
 +        context.stop(self)
 +
 +      case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) =>
 +        val fullId = appId + "/" + id
 +        logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores))
 +        listener.executorAdded(fullId, workerId, hostPort, cores, memory)
 +
 +      case ExecutorUpdated(id, state, message, exitStatus) =>
 +        val fullId = appId + "/" + id
 +        val messageText = message.map(s => " (" + s + ")").getOrElse("")
 +        logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText))
 +        if (ExecutorState.isFinished(state)) {
 +          listener.executorRemoved(fullId, message.getOrElse(""), exitStatus)
 +        }
 +
 +      case MasterChanged(masterUrl, masterWebUiUrl) =>
 +        logInfo("Master has changed, new master is at " + masterUrl)
 +        changeMaster(masterUrl)
 +        alreadyDisconnected = false
 +        sender ! MasterChangeAcknowledged(appId)
 +
 +      case DisassociatedEvent(_, address, _) if address == masterAddress =>
 +        logWarning(s"Connection to $address failed; waiting for master to reconnect...")
 +        markDisconnected()
 +
++      case AssociationErrorEvent(cause, _, address, _) if isPossibleMaster(address) =>
++        logWarning(s"Could not connect to $address: $cause")
++
 +      case StopAppClient =>
 +        markDead()
 +        sender ! true
 +        context.stop(self)
 +    }
 +
 +    /**
 +     * Notify the listener that we disconnected, if we hadn't already done so before.
 +     */
 +    def markDisconnected() {
 +      if (!alreadyDisconnected) {
 +        listener.disconnected()
 +        alreadyDisconnected = true
 +      }
 +    }
 +
 +    def markDead() {
 +      if (!alreadyDead) {
 +        listener.dead()
 +        alreadyDead = true
 +      }
 +    }
 +  }
 +
 +  def start() {
 +    // Just launch an actor; it will call back into the listener.
 +    actor = actorSystem.actorOf(Props(new ClientActor))
 +  }
 +
 +  def stop() {
 +    if (actor != null) {
 +      try {
-         val timeout = AkkaUtils.askTimeout
++        val timeout = AkkaUtils.askTimeout(conf)
 +        val future = actor.ask(StopAppClient)(timeout)
 +        Await.result(future, timeout)
 +      } catch {
 +        case e: TimeoutException =>
 +          logInfo("Stop request to Master timed out; it may already be shut down.")
 +      }
 +      actor = null
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0498f92/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index 9359bf1,ef649fd..62567a2
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@@ -45,11 -45,13 +45,13 @@@ private[spark] object TestClient 
  
    def main(args: Array[String]) {
      val url = args(0)
-     val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0)
+     val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0,
+       conf = new SparkConf)
      val desc = new ApplicationDescription(
-       "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored")
+       "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()),
+       "dummy-spark-home", "ignored")
      val listener = new TestListener
-     val client = new AppClient(actorSystem, Array(url), desc, listener)
 -    val client = new Client(actorSystem, Array(url), desc, listener, new SparkConf)
++    val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf)
      client.start()
      actorSystem.awaitTermination()
    }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0498f92/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 29f20da,7b696cf..93fa700
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@@ -30,25 -29,27 +30,28 @@@ import akka.pattern.as
  import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
  import akka.serialization.SerializationExtension
  
- import org.apache.spark.{Logging, SparkException}
 -import org.apache.spark.{SparkConf, SparkContext, Logging, SparkException}
 -import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
++
++import org.apache.spark.{SparkConf, Logging, SparkException}
 +import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState}
  import org.apache.spark.deploy.DeployMessages._
  import org.apache.spark.deploy.master.MasterMessages._
  import org.apache.spark.deploy.master.ui.MasterWebUI
  import org.apache.spark.metrics.MetricsSystem
  import org.apache.spark.util.{AkkaUtils, Utils}
 +import org.apache.spark.deploy.master.DriverState.DriverState
  
  private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
-   import context.dispatcher
+   import context.dispatcher   // to use Akka's scheduler.schedule()
+ 
+   val conf = new SparkConf
  
    val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
-   val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000
-   val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt
-   val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt
-   val RECOVERY_DIR = System.getProperty("spark.deploy.recoveryDirectory", "")
-   val RECOVERY_MODE = System.getProperty("spark.deploy.recoveryMode", "NONE")
+   val WORKER_TIMEOUT = conf.get("spark.worker.timeout", "60").toLong * 1000
+   val RETAINED_APPLICATIONS = conf.get("spark.deploy.retainedApplications", "200").toInt
+   val REAPER_ITERATIONS = conf.get("spark.dead.worker.persistence", "15").toInt
+   val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "")
+   val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE")
  
 -  var nextAppNumber = 0
    val workers = new HashSet[WorkerInfo]
    val idToWorker = new HashMap[String, WorkerInfo]
    val actorToWorker = new HashMap[ActorRef, WorkerInfo]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0498f92/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0498f92/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 2947ed1,fcaf4e9..2072f00
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@@ -26,11 -26,10 +26,11 @@@ import scala.concurrent.duration.
  
  import akka.actor._
  import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
 +
- import org.apache.spark.{Logging, SparkException}
+ import org.apache.spark.{Logging, SparkConf, SparkException}
  import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
  import org.apache.spark.deploy.DeployMessages._
 -import org.apache.spark.deploy.master.Master
 +import org.apache.spark.deploy.master.{DriverState, Master}
  import org.apache.spark.deploy.worker.ui.WorkerWebUI
  import org.apache.spark.metrics.MetricsSystem
  import org.apache.spark.util.{AkkaUtils, Utils}
@@@ -45,9 -44,8 +45,10 @@@ private[spark] class Worker
      cores: Int,
      memory: Int,
      masterUrls: Array[String],
 +    actorSystemName: String,
 +    actorName: String,
-     workDirPath: String = null)
+     workDirPath: String = null,
+     val conf: SparkConf)
    extends Actor with Logging {
    import context.dispatcher
  
@@@ -323,14 -276,16 +325,17 @@@ private[spark] object Worker 
    }
  
    def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int,
-     masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None)
-     : (ActorSystem, Int) = {
+       masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None)
+       : (ActorSystem, Int) =
+   {
      // The LocalSparkCluster runs multiple local sparkWorkerX actor systems
+     val conf = new SparkConf
      val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
 +    val actorName = "Worker"
-     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
+     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port,
+       conf = conf)
      actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
-       masterUrls, systemName, actorName, workDir), name = actorName)
 -      masterUrls, workDir, conf), name = "Worker")
++      masterUrls, workDir, conf), name = actorName)
      (actorSystem, boundPort)
    }
  

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0498f92/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0498f92/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index eb1199e,53a2b94..f9e43e0
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@@ -22,11 -22,10 +22,11 @@@ import java.nio.ByteBuffe
  import akka.actor._
  import akka.remote._
  
- import org.apache.spark.Logging
+ import org.apache.spark.{SparkConf, SparkContext, Logging}
  import org.apache.spark.TaskState.TaskState
 +import org.apache.spark.deploy.worker.WorkerWatcher
  import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
 -import org.apache.spark.util.{Utils, AkkaUtils}
 +import org.apache.spark.util.{AkkaUtils, Utils}
  
  private[spark] class CoarseGrainedExecutorBackend(
      driverUrl: String,

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0498f92/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 0615f7b,9858717..c8c77e9
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@@ -45,16 -45,16 +45,16 @@@ private[spark] class SparkDeploySchedul
  
      // The endpoint for executors to talk to us
      val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
-       System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
+       conf.get("spark.driver.host"),  conf.get("spark.driver.port"),
        CoarseGrainedSchedulerBackend.ACTOR_NAME)
 -    val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
 +    val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}")
      val command = Command(
        "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
      val sparkHome = sc.getSparkHome().getOrElse(null)
-     val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
+     val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome,
          "http://" + sc.ui.appUIAddress)
  
-     client = new AppClient(sc.env.actorSystem, masters, appDesc, this)
 -    client = new Client(sc.env.actorSystem, masters, appDesc, this, conf)
++    client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf)
      client.start()
    }
  

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0498f92/docs/spark-standalone.md
----------------------------------------------------------------------


[08/37] git commit: Small fix from rebase

Posted by pw...@apache.org.
Small fix from rebase


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

Branch: refs/heads/master
Commit: 18ad419b521bab4c5ffd8761652905fdc116a163
Parents: 55f8338
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Dec 25 01:22:38 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Dec 25 01:22:38 2013 -0800

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/18ad419b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
index 737ba09..4779c75 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
@@ -180,7 +180,7 @@ private[spark] class AppClient(
     if (actor != null) {
       try {
         val timeout = AkkaUtils.askTimeout
-        val future = actor.ask(StopClient)(timeout)
+        val future = actor.ask(StopAppClient)(timeout)
         Await.result(future, timeout)
       } catch {
         case e: TimeoutException =>


[09/37] git commit: Removing un-used variable

Posted by pw...@apache.org.
Removing un-used variable


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

Branch: refs/heads/master
Commit: bbc362833b3bc34014a13be0592deca39cfd88bd
Parents: 18ad419
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Dec 25 01:38:57 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Dec 25 01:38:57 2013 -0800

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/deploy/master/DriverState.scala    | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/bbc36283/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
index 230dab1..93b2607 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
@@ -29,6 +29,4 @@ private[spark] object DriverState extends Enumeration {
   // KILLED: A user manually killed this driver
   // FAILED: Unable to run due to an unrecoverable error (e.g. missing jar file)
   val SUBMITTED, RUNNING, FINISHED, RELAUNCHING, UNKNOWN, KILLED, FAILED = Value
-
-  val MAX_NUM_RETRY = 10
 }


[12/37] git commit: Removing accidental file

Posted by pw...@apache.org.
Removing accidental file


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

Branch: refs/heads/master
Commit: a97ad55c45f5903c85c7c15b27177316473f2c0c
Parents: 5938cfc
Author: Patrick Wendell <pw...@gmail.com>
Authored: Thu Dec 26 12:11:28 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Thu Dec 26 12:11:28 2013 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/deploy/worker/DriverRunner.scala  | 3 +--
 .../scala/org/apache/spark/deploy/worker/DriverWrapper.scala | 8 --------
 2 files changed, 1 insertion(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a97ad55c/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index d13e651..402ad53 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -136,8 +136,7 @@ private[spark] class DriverRunner(
   }
 
   /** Continue launching the supplied command until it exits zero or is killed. */
-  def runCommandWithRetry(command: Seq[String], envVars: Seq[(String, String)], baseDir: File)
-      : Unit = {
+  def runCommandWithRetry(command: Seq[String], envVars: Seq[(String, String)], baseDir: File) {
     // Time to wait between submission retries.
     var waitSeconds = 1
     var cleanExit = false

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a97ad55c/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
deleted file mode 100644
index 92fb084..0000000
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
+++ /dev/null
@@ -1,8 +0,0 @@
-package org.apache.spark.deploy.worker
-
-object DriverWrapper {
-  def main(args: Array[String]) {
-    val c = Console.readChar()
-    println(s"Char: $c")
-  }
-}


[07/37] git commit: Import clean-up (yay Aaron)

Posted by pw...@apache.org.
Import clean-up (yay Aaron)


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

Branch: refs/heads/master
Commit: b2b7514ba31be9c18f107ed466849e6360ded7c6
Parents: d5f23e0
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Dec 25 00:38:31 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Dec 25 01:19:25 2013 -0800

----------------------------------------------------------------------
 .../org/apache/spark/deploy/DeployMessage.scala |  5 ++---
 .../spark/deploy/client/DriverClient.scala      | 22 ++++++++++----------
 .../apache/spark/deploy/master/DriverInfo.scala |  5 ++---
 .../master/FileSystemPersistenceEngine.scala    |  2 --
 .../org/apache/spark/deploy/master/Master.scala |  3 ++-
 .../apache/spark/deploy/master/WorkerInfo.scala |  4 +++-
 .../spark/deploy/master/ui/IndexPage.scala      |  4 ++--
 .../spark/deploy/worker/DriverRunner.scala      | 10 ++++-----
 .../org/apache/spark/deploy/worker/Worker.scala |  6 +++---
 .../spark/deploy/worker/ui/IndexPage.scala      | 10 +++------
 10 files changed, 33 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 332c7e8..82bb33a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -20,12 +20,11 @@ package org.apache.spark.deploy
 import scala.collection.immutable.List
 
 import org.apache.spark.deploy.ExecutorState.ExecutorState
-import org.apache.spark.deploy.master.{DriverInfo, WorkerInfo, ApplicationInfo}
+import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
+import org.apache.spark.deploy.master.DriverState.DriverState
 import org.apache.spark.deploy.master.RecoveryState.MasterState
 import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
 import org.apache.spark.util.Utils
-import org.apache.spark.deploy.master.DriverState.DriverState
-
 
 private[deploy] sealed trait DeployMessage extends Serializable
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index dd62172..9c0a626 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -17,21 +17,21 @@
 
 package org.apache.spark.deploy.client
 
+import java.util.concurrent.TimeUnit
+
+import scala.concurrent.Await
+import scala.concurrent.duration.{Duration, FiniteDuration}
+
 import akka.actor._
-import akka.remote.{RemotingLifecycleEvent}
+import akka.actor.Actor.emptyBehavior
+import akka.pattern.ask
+import akka.remote.RemotingLifecycleEvent
 
-import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.Logging
 import org.apache.spark.deploy.{DeployMessage, DriverDescription}
 import org.apache.spark.deploy.DeployMessages._
-import org.apache.spark.deploy.master.{MasterArguments, Master}
-import akka.pattern.ask
-
-import org.apache.spark.util.{Utils, AkkaUtils}
-import scala.concurrent.duration.{FiniteDuration, Duration}
-import java.util.concurrent.TimeUnit
-import akka.util.Timeout
-import scala.concurrent.Await
-import akka.actor.Actor.emptyBehavior
+import org.apache.spark.deploy.master.Master
+import org.apache.spark.util.{AkkaUtils, Utils}
 
 /**
  * Actor that sends a single message to the standalone master and then shuts down.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
index 69d150a..052c474 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
@@ -17,10 +17,9 @@
 
 package org.apache.spark.deploy.master
 
-import org.apache.spark.deploy.{DriverDescription, ApplicationDescription}
 import java.util.Date
-import akka.actor.ActorRef
-import scala.collection.mutable
+
+import org.apache.spark.deploy.DriverDescription
 
 private[spark] class DriverInfo(
     val startTime: Long,

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
index 44a046b..74bb9eb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
@@ -19,8 +19,6 @@ package org.apache.spark.deploy.master
 
 import java.io._
 
-import scala.Serializable
-
 import akka.serialization.Serialization
 import org.apache.spark.Logging
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 939e000..f5d6fda 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -19,6 +19,7 @@ package org.apache.spark.deploy.master
 
 import java.text.SimpleDateFormat
 import java.util.Date
+import java.util.concurrent.TimeUnit
 
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 import scala.concurrent.Await
@@ -30,7 +31,7 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
 import akka.serialization.SerializationExtension
 
 import org.apache.spark.{Logging, SparkException}
-import org.apache.spark.deploy.{DriverDescription, ApplicationDescription, ExecutorState}
+import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.MasterMessages._
 import org.apache.spark.deploy.master.ui.MasterWebUI

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index 6e5177b..28cd463 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -17,8 +17,10 @@
 
 package org.apache.spark.deploy.master
 
-import akka.actor.ActorRef
 import scala.collection.mutable
+
+import akka.actor.ActorRef
+
 import org.apache.spark.util.Utils
 
 private[spark] class WorkerInfo(

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index 24d10ce..6a99d7a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.deploy.master.ui
 
 import scala.concurrent.Await
+import scala.concurrent.duration._
 import scala.xml.Node
 
 import akka.pattern.ask
@@ -26,8 +27,7 @@ import net.liftweb.json.JsonAST.JValue
 
 import org.apache.spark.deploy.{DeployWebUI, JsonProtocol}
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
-import org.apache.spark.deploy.JsonProtocol
-import org.apache.spark.deploy.master.{DriverInfo, ApplicationInfo, WorkerInfo}
+import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
 import org.apache.spark.ui.UIUtils
 import org.apache.spark.util.Utils
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index ba13f22..b030d60 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -19,17 +19,17 @@ package org.apache.spark.deploy.worker
 
 import java.io._
 
+import akka.actor.ActorRef
 import com.google.common.base.Charsets
 import com.google.common.io.Files
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileUtil, Path}
 
 import org.apache.spark.Logging
-import org.apache.spark.util.Utils
-import org.apache.hadoop.fs.{FileUtil, Path}
-import org.apache.hadoop.conf.Configuration
-import akka.actor.{ActorRef, ActorSelection}
+import org.apache.spark.deploy.DriverDescription
 import org.apache.spark.deploy.DeployMessages.DriverStateChanged
 import org.apache.spark.deploy.master.DriverState
-import org.apache.spark.deploy.DriverDescription
+import org.apache.spark.util.Utils
 
 /**
  * Manages the execution of one driver process.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 3c5159a..b6a84fc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -25,15 +25,15 @@ import scala.collection.mutable.HashMap
 import scala.concurrent.duration._
 
 import akka.actor._
-import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent}
+import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
 
-import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.{Logging, SparkException}
 import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.{DriverState, Master}
 import org.apache.spark.deploy.worker.ui.WorkerWebUI
 import org.apache.spark.metrics.MetricsSystem
-import org.apache.spark.util.{Utils, AkkaUtils}
+import org.apache.spark.util.{AkkaUtils, Utils}
 
 /**
   * @param masterUrls Each url should look like spark://host:port.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2b7514b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index 35e8d58..c8cafac 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -17,15 +17,12 @@
 
 package org.apache.spark.deploy.worker.ui
 
-import javax.servlet.http.HttpServletRequest
-
-import scala.xml.Node
-
-import scala.concurrent.duration._
 import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.xml.Node
 
 import akka.pattern.ask
-
+import javax.servlet.http.HttpServletRequest
 import net.liftweb.json.JsonAST.JValue
 
 import org.apache.spark.deploy.JsonProtocol
@@ -34,7 +31,6 @@ import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
 import org.apache.spark.ui.UIUtils
 import org.apache.spark.util.Utils
 
-
 private[spark] class IndexPage(parent: WorkerWebUI) {
   val workerActor = parent.worker.self
   val worker = parent.worker


[05/37] git commit: Minor bug fix

Posted by pw...@apache.org.
Minor bug 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/55f83380
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/55f83380
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/55f83380

Branch: refs/heads/master
Commit: 55f833803a8e5755eb01b99b976752e5c6bf14e2
Parents: c9c0f74
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Dec 25 01:00:53 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Dec 25 01:19:25 2013 -0800

----------------------------------------------------------------------
 .../org/apache/spark/deploy/client/DriverClientArguments.scala  | 5 +++++
 .../scala/org/apache/spark/deploy/worker/DriverRunner.scala     | 2 +-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/55f83380/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
index 618467c..60e6549 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
@@ -58,7 +58,12 @@ private[spark] class DriverClientArguments(args: Array[String]) {
 
     case ("--environment-variable" | "-e") :: value :: tail =>
       val parts = value.split("=")
+      if (parts.length != 2) {
+        println(s"Error - invalid environment variable (expecting K=V): $value")
+        printUsageAndExit(1)
+      }
       _driverEnvVars += ((parts(0), parts(1)))
+      parse(tail)
 
     case ("--help" | "-h") :: tail =>
       printUsageAndExit(0)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/55f83380/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 28d4297..b9b6341 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -53,7 +53,7 @@ private[spark] class DriverRunner(
         try {
           val driverDir = createWorkingDirectory()
           val localJarFilename = downloadUserJar(driverDir)
-          val command = Seq("java") ++ driverDesc.javaOptions ++ Seq(s"-Xmx${driverDesc.mem}m")
+          val command = Seq("java") ++ driverDesc.javaOptions ++ Seq(s"-Xmx${driverDesc.mem}m") ++
             Seq("-cp", localJarFilename) ++ Seq(driverDesc.mainClass) ++ driverDesc.options
           runCommandWithRetry(command, driverDesc.envVars, driverDir)
         }


[20/37] git commit: Adding driver ID to submission response

Posted by pw...@apache.org.
Adding driver ID to submission response


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

Branch: refs/heads/master
Commit: faefea3fd8e24da059f1e35f618310aeabe2d847
Parents: 6ffa9bb
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sun Dec 29 11:31:10 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sun Dec 29 11:31:10 2013 -0800

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/deploy/client/DriverClient.scala   | 2 +-
 core/src/main/scala/org/apache/spark/deploy/master/Master.scala    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/faefea3f/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index 6257303..a5f5763 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -43,7 +43,7 @@ class DriverActor(master: String, response: Promise[(Boolean, String)]) extends
       response.success((success, message))
     }
 
-    // Relay all other messages to the server.
+    // Relay all other messages to the master.
     case message => {
       logInfo(s"Sending message to master $master...")
       val masterActor = context.actorSelection(Master.toAkkaUrl(master))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/faefea3f/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index efb9bf4..b8655c7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -189,7 +189,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         // TODO: It might be good to instead have the submission client poll the master to determine
         //       the current status of the driver. For now it's simply "fire and forget".
 
-        sender ! SubmitDriverResponse(true, "Driver successfully submitted")
+        sender ! SubmitDriverResponse(true, s"Driver successfully submitted as ${driver.id}")
       }
     }
 


[27/37] git commit: Some doc fixes

Posted by pw...@apache.org.
Some doc fixes


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

Branch: refs/heads/master
Commit: b72cceba2727586c1e1f89c58b66417628e1afa7
Parents: 6a3daea
Author: Patrick Wendell <pw...@gmail.com>
Authored: Mon Jan 6 22:05:53 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Mon Jan 6 22:05:53 2014 -0800

----------------------------------------------------------------------
 docs/spark-standalone.md | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b72cceba/docs/spark-standalone.md
----------------------------------------------------------------------
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index f426db0..7da6474 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -157,8 +157,7 @@ You may also run your application entirely inside of the cluster by submitting y
        [application-options]
 
     cluster-url: The URL of the master node.
-    application-jar-url: Path to a bundled jar including your application and all dependencies.
-                         Accepts hdfs://, file://, and http:// paths.
+    application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be visible from inside of your cluster, for instance, in an HDFS directory. 
     main-class: The entry point for your application.
 
     Client Options:
@@ -170,7 +169,7 @@ Keep in mind that your driver program will be executed on a remote worker machin
 
  * _Environment variables_: These will be captured from the environment in which you launch the client and applied when launching the driver program.
  * _Java options_: You can add java options by setting `SPARK_JAVA_OPTS` in the environment in which you launch the submission client.
-  * _Dependencies_: You'll still need to call `sc.addJar` inside of your driver program to add your application jar and any dependencies. If you submit a local application jar to the client (e.g one with a `file://` URL), it will be uploaded into the working directory of your driver program. Then, you can add it using `sc.addJar("jar-name.jar")`.
+ * _Dependencies_: You'll still need to call `sc.addJar` inside of your program to make your bundled application jar visible on all worker nodes.
 
 Once you submit a driver program, it will appear in the cluster management UI at port 8080 and
 be assigned an identifier. If you'd like to prematurely terminate the program, you can do so using


[02/37] git commit: Adding scheduling and reporting based on cores

Posted by pw...@apache.org.
Adding scheduling and reporting based on cores


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

Branch: refs/heads/master
Commit: d5f23e0083b8b00109eb466ef5fb74558dcbc040
Parents: 760823d
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Dec 25 00:24:00 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Dec 25 01:19:01 2013 -0800

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 2 +-
 .../main/scala/org/apache/spark/deploy/master/WorkerInfo.scala  | 4 ++--
 .../scala/org/apache/spark/deploy/master/ui/IndexPage.scala     | 5 ++++-
 .../scala/org/apache/spark/deploy/worker/DriverRunner.scala     | 4 ++--
 core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 2 +-
 .../scala/org/apache/spark/deploy/worker/ui/IndexPage.scala     | 5 ++++-
 6 files changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d5f23e00/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 9bfacfc..939e000 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -425,7 +425,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     // First schedule drivers, they take strict precedence over applications
     for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) {
       for (driver <- Seq(waitingDrivers: _*)) {
-        if (worker.memoryFree > driver.desc.mem) {
+        if (worker.memoryFree > driver.desc.mem && worker.coresFree > driver.desc.cores) {
           launchDriver(worker, driver)
           waitingDrivers -= driver
         }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d5f23e00/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index 27c2ff4..6e5177b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -88,13 +88,13 @@ private[spark] class WorkerInfo(
   def addDriver(driver: DriverInfo) {
     drivers(driver.id) = driver
     memoryUsed += driver.desc.mem
-    coresUsed += 1
+    coresUsed += driver.desc.cores
   }
 
   def removeDriver(driver: DriverInfo) {
     drivers -= driver.id
     memoryUsed -= driver.desc.mem
-    coresUsed -= 1
+    coresUsed -= driver.desc.cores
   }
 
   def webUiAddress : String = {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d5f23e00/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index 13903b4..24d10ce 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -57,7 +57,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
     val completedApps = state.completedApps.sortBy(_.endTime).reverse
     val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps)
 
-    val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Memory", "Main Class")
+    val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory", "Main Class")
     val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse
     val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers)
     val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse
@@ -166,6 +166,9 @@ private[spark] class IndexPage(parent: MasterWebUI) {
       <td>{driver.submitDate}</td>
       <td>{driver.worker.map(w => w.id.toString).getOrElse("None")}</td>
       <td>{driver.state}</td>
+      <td sorttable_customkey={driver.desc.cores.toString}>
+        {driver.desc.cores.toString}
+      </td>
       <td sorttable_customkey={driver.desc.mem.toString}>
         {Utils.megabytesToString(driver.desc.mem.toLong)}
       </td>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d5f23e00/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 41a089a..ba13f22 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -53,8 +53,8 @@ private[spark] class DriverRunner(
         try {
           val driverDir = createWorkingDirectory()
           val localJarFilename = downloadUserJar(driverDir)
-          val command = Seq("java") ++ driverDesc.javaOptions ++ Seq("-cp", localJarFilename) ++
-            Seq(driverDesc.mainClass) ++ driverDesc.options
+          val command = Seq("java") ++ driverDesc.javaOptions ++ Seq(s"-Xmx${driverDesc.mem}m")
+            Seq("-cp", localJarFilename) ++ Seq(driverDesc.mainClass) ++ driverDesc.options
           runCommandWithRetry(command, driverDesc.envVars, driverDir)
         }
         catch {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d5f23e00/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index dd6783a..3c5159a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -279,7 +279,7 @@ private[spark] class Worker(
       }
       val driver = drivers(driverId)
       memoryUsed -= driver.driverDesc.mem
-      coresUsed -= 1
+      coresUsed -= driver.driverDesc.cores
       drivers -= driverId
       finishedDrivers(driverId) = driver
     }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d5f23e00/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index 2c37b71..35e8d58 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -56,7 +56,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
     val finishedExecutorTable =
       UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors)
 
-    val driverHeaders = Seq("DriverID", "Main Class", "Memory", "Logs")
+    val driverHeaders = Seq("DriverID", "Main Class", "Cores", "Memory", "Logs")
     val runningDriverTable =
       UIUtils.listingTable(driverHeaders, driverRow, workerState.drivers)
     def finishedDriverTable =
@@ -138,6 +138,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
     <tr>
       <td>{driver.driverId}</td>
       <td>{driver.driverDesc.mainClass}</td>
+      <td sorttable_customkey={driver.driverDesc.cores.toString}>
+        {driver.driverDesc.cores.toString}
+      </td>
       <td sorttable_customkey={driver.driverDesc.mem.toString}>
         {Utils.megabytesToString(driver.driverDesc.mem)}
       </td>


[16/37] git commit: Intermediate clean-up of tests to appease jenkins

Posted by pw...@apache.org.
Intermediate clean-up of tests to appease jenkins


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

Branch: refs/heads/master
Commit: 55c8bb741c9e3780a63b88ff24146b1d81663142
Parents: 5c1b4f6
Author: Patrick Wendell <pw...@gmail.com>
Authored: Thu Dec 26 15:43:15 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Thu Dec 26 15:43:15 2013 -0800

----------------------------------------------------------------------
 .../apache/spark/deploy/JsonProtocolSuite.scala | 35 ++++++++++++++------
 1 file changed, 25 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/55c8bb74/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index 0b38e23..485f688 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -25,8 +25,8 @@ import net.liftweb.json.JsonAST.JValue
 import org.scalatest.FunSuite
 
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
-import org.apache.spark.deploy.master.{ApplicationInfo, RecoveryState, WorkerInfo}
-import org.apache.spark.deploy.worker.ExecutorRunner
+import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo}
+import org.apache.spark.deploy.worker.{ExecutorRunner, DriverRunner}
 
 class JsonProtocolSuite extends FunSuite {
   test("writeApplicationInfo") {
@@ -50,11 +50,13 @@ class JsonProtocolSuite extends FunSuite {
   }
 
   test("writeMasterState") {
-    val workers = Array[WorkerInfo](createWorkerInfo(), createWorkerInfo())
-    val activeApps = Array[ApplicationInfo](createAppInfo())
+    val workers = Array(createWorkerInfo(), createWorkerInfo())
+    val activeApps = Array(createAppInfo())
     val completedApps = Array[ApplicationInfo]()
+    val activeDrivers = Array(createDriverInfo())
+    val completedDrivers = Array(createDriverInfo())
     val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps,
-      RecoveryState.ALIVE)
+      activeDrivers, completedDrivers, RecoveryState.ALIVE)
     val output = JsonProtocol.writeMasterState(stateResponse)
     assertValidJson(output)
   }
@@ -62,26 +64,39 @@ class JsonProtocolSuite extends FunSuite {
   test("writeWorkerState") {
     val executors = List[ExecutorRunner]()
     val finishedExecutors = List[ExecutorRunner](createExecutorRunner(), createExecutorRunner())
+    val drivers = List(createDriverRunner())
+    val finishedDrivers = List(createDriverRunner(), createDriverRunner())
     val stateResponse = new WorkerStateResponse("host", 8080, "workerId", executors,
-      finishedExecutors, "masterUrl", 4, 1234, 4, 1234, "masterWebUiUrl")
+      finishedExecutors, drivers, finishedDrivers, "masterUrl", 4, 1234, 4, 1234, "masterWebUiUrl")
     val output = JsonProtocol.writeWorkerState(stateResponse)
     assertValidJson(output)
   }
 
-  def createAppDesc() : ApplicationDescription = {
+  def createAppDesc(): ApplicationDescription = {
     val cmd = new Command("mainClass", List("arg1", "arg2"), Map())
     new ApplicationDescription("name", 4, 1234, cmd, "sparkHome", "appUiUrl")
   }
-  def createAppInfo() : ApplicationInfo = {
+  def createAppInfo(): ApplicationInfo = {
     new ApplicationInfo(3, "id", createAppDesc(), new Date(123456789), null, "appUriStr")
   }
-  def createWorkerInfo() : WorkerInfo = {
+
+  def createDriverDesc() = new DriverDescription(
+    "hdfs://some-dir/some.jar", "org.apache.spark.FakeClass", 100, 3,
+    Seq("--some-config", "val", "--other-config", "val"), Seq("-Dsystem.property=someValue"),
+    Seq(("K1", "V1"), ("K2", "V2"))
+  )
+  def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", createDriverDesc(), new Date())
+
+  def createWorkerInfo(): WorkerInfo = {
     new WorkerInfo("id", "host", 8080, 4, 1234, null, 80, "publicAddress")
   }
-  def createExecutorRunner() : ExecutorRunner = {
+  def createExecutorRunner(): ExecutorRunner = {
     new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host",
       new File("sparkHome"), new File("workDir"), ExecutorState.RUNNING)
   }
+  def createDriverRunner(): DriverRunner = {
+    new DriverRunner("driverId", new File("workDir"), createDriverDesc(), null)
+  }
 
   def assertValidJson(json: JValue) {
     try {


[24/37] git commit: Changes based on review feedback.

Posted by pw...@apache.org.
Changes based on review feedback.


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

Branch: refs/heads/master
Commit: f236ddd1a245a587d5ee331fb67cf41456ed383c
Parents: 7a99702
Author: Patrick Wendell <pw...@gmail.com>
Authored: Thu Jan 2 18:10:37 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Mon Jan 6 17:15:52 2014 -0800

----------------------------------------------------------------------
 .../spark/deploy/client/DriverClient.scala      |  2 +-
 .../deploy/client/DriverClientArguments.scala   | 26 ++++++++++++--------
 .../spark/deploy/worker/CommandUtils.scala      |  2 +-
 .../spark/deploy/worker/DriverRunner.scala      | 15 ++++++++---
 .../spark/deploy/worker/DriverWrapper.scala     |  2 +-
 .../org/apache/spark/deploy/worker/Worker.scala |  7 +++---
 .../cluster/SparkDeploySchedulerBackend.scala   |  2 +-
 7 files changed, 34 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f236ddd1/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index 8a4cdf0..e319e75 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -62,7 +62,7 @@ object DriverClient extends Logging {
 
     // TODO: See if we can initialize akka so return messages are sent back using the same TCP
     //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(
+    val (actorSystem, _) = AkkaUtils.createActorSystem(
       "driverClient", Utils.localHostName(), 0)
     val master = driverArgs.master
     val response = promise[(Boolean, String)]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f236ddd1/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
index 6a15422..d9e1c8a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
@@ -23,6 +23,9 @@ import scala.collection.mutable.ListBuffer
  * Command-line parser for the driver client.
  */
 private[spark] class DriverClientArguments(args: Array[String]) {
+  val defaultCores = 1
+  val defaultMemory = 512
+
   var cmd: String = "" // 'launch' or 'kill'
 
   // launch parameters
@@ -30,8 +33,8 @@ private[spark] class DriverClientArguments(args: Array[String]) {
   var jarUrl: String = ""
   var mainClass: String = ""
   var supervise: Boolean = false
-  var memory: Int = 512
-  var cores: Int = 1
+  var memory: Int = defaultMemory
+  var cores: Int = defaultCores
   private var _driverOptions = ListBuffer[String]()
   def driverOptions = _driverOptions.toSeq
 
@@ -78,14 +81,17 @@ private[spark] class DriverClientArguments(args: Array[String]) {
   def printUsageAndExit(exitCode: Int) {
     // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars
     //       separately similar to in the YARN client.
-    System.err.println(
-      "usage: DriverClient [options] launch <active-master> <jar-url> <main-class> " +
-        "[driver options]\n" +
-      "usage: DriverClient kill <active-master> <driver-id>\n\n" +
-      "Options:\n" +
-      "  -c CORES, --cores CORES                Number of cores to request \n" +
-      "  -m MEMORY, --memory MEMORY             Megabytes of memory to request\n" +
-      "  -s, --supervise                        Whether to restart the driver on failure\n")
+    val usage =
+      s"""
+        |Usage: DriverClient [options] launch <active-master> <jar-url> <main-class> [driver options]
+        |Usage: DriverClient kill <active-master> <driver-id>
+        |
+        |Options:
+        |   -c CORES, --cores CORES        Number of cores to request (default: $defaultCores)
+        |   -m MEMORY, --memory MEMORY     Megabytes of memory to request (default: $defaultMemory)
+        |   -s, --supervise                Whether to restart the driver on failure
+      """.stripMargin
+    System.err.println(usage)
     System.exit(exitCode)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f236ddd1/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
index 785aecf..7507bf8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
@@ -21,7 +21,7 @@ object CommandUtils extends Logging {
   }
 
   private def getEnv(key: String, command: Command): Option[String] =
-    command.environment.get(key).orElse(Option(getenv(key)))
+    command.environment.get(key).orElse(Option(System.getenv(key)))
 
   /**
    * Attention: this must always be aligned with the environment variables in the run scripts and

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f236ddd1/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index e8ae2d3..f726089 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -119,15 +119,14 @@ private[spark] class DriverRunner(
     val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path
     val jarFileSystem = jarPath.getFileSystem(emptyConf)
 
-    val destPath = new Path(driverDir.getAbsolutePath())
-    val destFileSystem = destPath.getFileSystem(emptyConf)
+    val destPath = new File(driverDir.getAbsolutePath, jarPath.getName)
     val jarFileName = jarPath.getName
     val localJarFile = new File(driverDir, jarFileName)
     val localJarFilename = localJarFile.getAbsolutePath
 
     if (!localJarFile.exists()) { // May already exist if running multiple workers on one node
       logInfo(s"Copying user jar $jarPath to $destPath")
-      FileUtil.copy(jarFileSystem, jarPath, destFileSystem, destPath, false, false, emptyConf)
+      FileUtil.copy(jarFileSystem, jarPath, destPath, false, emptyConf)
     }
 
     if (!localJarFile.exists()) { // Verify copy succeeded
@@ -140,8 +139,12 @@ private[spark] class DriverRunner(
   /** Launch the supplied command. */
   private def runCommand(command: Seq[String], envVars: Map[String, String], baseDir: File,
       supervise: Boolean) {
+
     // Time to wait between submission retries.
     var waitSeconds = 1
+    // A run of this many seconds resets the exponential back-off.
+    val successfulRunDuration = 1
+
     var keepTrying = !killed
 
     while (keepTrying) {
@@ -161,11 +164,15 @@ private[spark] class DriverRunner(
         val stderr = new File(baseDir, "stderr")
         val header = "Launch Command: %s\n%s\n\n".format(
           command.mkString("\"", "\" \"", "\""), "=" * 40)
-        Files.write(header, stderr, Charsets.UTF_8)
+        Files.append(header, stderr, Charsets.UTF_8)
         CommandUtils.redirectStream(process.get.getErrorStream, stderr)
       }
 
+      val processStart = System.currentTimeMillis()
       val exitCode = process.get.waitFor()
+      if (System.currentTimeMillis() - processStart > successfulRunDuration * 1000) {
+        waitSeconds = 1
+      }
 
       if (supervise && exitCode != 0 && !killed) {
         waitSeconds = waitSeconds * 2 // exponential back-off

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f236ddd1/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
index 8c13b10..2deb21a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -11,7 +11,7 @@ object DriverWrapper {
   def main(args: Array[String]) {
     args.toList match {
       case workerUrl :: mainClass :: extraArgs =>
-        val (actorSystem, boundPort) = AkkaUtils.createActorSystem("Driver",
+        val (actorSystem, _) = AkkaUtils.createActorSystem("Driver",
           Utils.localHostName(), 0)
         actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher")
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f236ddd1/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 4e23e0d..2947ed1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -260,8 +260,8 @@ private[spark] class Worker(
 
     case KillDriver(driverId) => {
       logInfo(s"Asked to kill driver $driverId")
-      drivers.find(_._1 == driverId) match {
-        case Some((id, runner)) =>
+      drivers.get(driverId) match {
+        case Some(runner) =>
           runner.kill()
         case None =>
           logError(s"Asked to kill unknown driver $driverId")
@@ -280,8 +280,7 @@ private[spark] class Worker(
       masterLock.synchronized {
         master ! DriverStateChanged(driverId, state, exception)
       }
-      val driver = drivers(driverId)
-      drivers -= driverId
+      val driver = drivers.remove(driverId).get
       finishedDrivers(driverId) = driver
       memoryUsed -= driver.driverDesc.mem
       coresUsed -= driver.driverDesc.cores

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f236ddd1/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 921b887..0615f7b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -47,7 +47,7 @@ private[spark] class SparkDeploySchedulerBackend(
     val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
       System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
       CoarseGrainedSchedulerBackend.ACTOR_NAME)
-    val args = Seq(driverUrl, "{{WORKER_URL}}", "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
+    val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}")
     val command = Command(
       "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
     val sparkHome = sc.getSparkHome().getOrElse(null)


[10/37] git commit: Refactor DriverClient to be more Actor-based

Posted by pw...@apache.org.
Refactor DriverClient to be more Actor-based


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

Branch: refs/heads/master
Commit: 61372b11f4a4460b8ade8997d7478234bba64f7e
Parents: bbc3628
Author: Aaron Davidson <aa...@databricks.com>
Authored: Wed Dec 25 10:55:09 2013 -0800
Committer: Aaron Davidson <aa...@databricks.com>
Committed: Wed Dec 25 10:55:25 2013 -0800

----------------------------------------------------------------------
 .../spark/deploy/client/DriverClient.scala      | 93 +++++++-------------
 1 file changed, 31 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/61372b11/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index 28c851b..d2f3c09 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -17,78 +17,35 @@
 
 package org.apache.spark.deploy.client
 
-import java.util.concurrent.TimeUnit
-
-import scala.concurrent.Await
-import scala.concurrent.duration.{Duration, FiniteDuration}
+import scala.concurrent._
 
 import akka.actor._
-import akka.actor.Actor.emptyBehavior
-import akka.pattern.ask
-import akka.remote.RemotingLifecycleEvent
 
 import org.apache.spark.Logging
-import org.apache.spark.deploy.{DeployMessage, DriverDescription}
+import org.apache.spark.deploy.DriverDescription
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.util.{AkkaUtils, Utils}
 
 /**
- * Actor that sends a single message to the standalone master and then shuts down.
+ * Actor that sends a single message to the standalone master and returns the response in the
+ * given promise.
  */
-private[spark] abstract class SingleMessageClient(
-    actorSystem: ActorSystem, master: String, message: DeployMessage)
-  extends Logging {
-
-  // Concrete child classes must implement
-  def handleResponse(response: Any)
-
-  var actor: ActorRef = actorSystem.actorOf(Props(new DriverActor()))
-
-  class DriverActor extends Actor with Logging {
-    override def preStart() {
-      context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
-      logInfo("Sending message to master " + master + "...")
-      val masterActor = context.actorSelection(Master.toAkkaUrl(master))
-      val timeoutDuration: FiniteDuration = Duration.create(
-        System.getProperty("spark.akka.askTimeout", "10").toLong, TimeUnit.SECONDS)
-      val submitFuture = masterActor.ask(message)(timeoutDuration)
-      handleResponse(Await.result(submitFuture, timeoutDuration))
-      actorSystem.stop(actor)
-      actorSystem.shutdown()
+class DriverActor(master: String, response: Promise[(Boolean, String)]) extends Actor with Logging {
+  override def receive = {
+    case SubmitDriverResponse(success, message) => {
+      response.success((success, message))
     }
 
-    override def receive = emptyBehavior
-  }
-}
-
-/**
- * Submits a driver to the master.
- */
-private[spark] class SubmissionClient(actorSystem: ActorSystem, master: String,
-    driverDescription: DriverDescription)
-    extends SingleMessageClient(actorSystem, master, RequestSubmitDriver(driverDescription)) {
-
-  override def handleResponse(response: Any) {
-    val resp = response.asInstanceOf[SubmitDriverResponse]
-    if (!resp.success) {
-      logError(s"Error submitting driver to $master")
-      logError(resp.message)
+    case KillDriverResponse(success, message) => {
+      response.success((success, message))
     }
-  }
-}
 
-/**
- * Terminates a client at the master.
- */
-private[spark] class TerminationClient(actorSystem: ActorSystem, master: String, driverId: String)
-    extends SingleMessageClient(actorSystem, master, RequestKillDriver(driverId)) {
-
-  override def handleResponse(response: Any) {
-    val resp = response.asInstanceOf[KillDriverResponse]
-    if (!resp.success) {
-      logError(s"Error terminating $driverId at $master")
-      logError(resp.message)
+    // Relay all other messages to the server.
+    case message => {
+      logInfo(s"Sending message to master $master...")
+      val masterActor = context.actorSelection(Master.toAkkaUrl(master))
+      masterActor ! message
     }
   }
 }
@@ -96,7 +53,7 @@ private[spark] class TerminationClient(actorSystem: ActorSystem, master: String,
 /**
  * Executable utility for starting and terminating drivers inside of a standalone cluster.
  */
-object DriverClient {
+object DriverClient extends Logging {
 
   def main(args: Array[String]) {
     val driverArgs = new DriverClientArguments(args)
@@ -105,6 +62,9 @@ object DriverClient {
     //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(
       "driverClient", Utils.localHostName(), 0)
+    val master = driverArgs.master
+    val response = promise[(Boolean, String)]
+    val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response)))
 
     driverArgs.cmd match {
       case "launch" =>
@@ -116,13 +76,22 @@ object DriverClient {
           driverArgs.driverOptions,
           driverArgs.driverJavaOptions,
           driverArgs.driverEnvVars)
-        val client = new SubmissionClient(actorSystem, driverArgs.master, driverDescription)
+        driver ! RequestSubmitDriver(driverDescription)
 
       case "kill" =>
-        val master = driverArgs.master
         val driverId = driverArgs.driverId
-        val client = new TerminationClient(actorSystem, master, driverId)
+        driver ! RequestKillDriver(driverId)
     }
+
+    val (success, message) =
+      try {
+        Await.result(response.future, AkkaUtils.askTimeout)
+      } catch {
+        case e: TimeoutException => (false, s"Master $master failed to respond in time")
+      }
+    if (success) logInfo(message) else logError(message)
+    actorSystem.stop(driver)
+    actorSystem.shutdown()
     actorSystem.awaitTermination()
   }
 }


[04/37] git commit: Initial cut at driver submission.

Posted by pw...@apache.org.
Initial cut at driver submission.


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

Branch: refs/heads/master
Commit: 6a4acc4c2d5c510cc76049dd8727cec76a2173e8
Parents: 1070b56
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sat Dec 21 21:08:13 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Dec 25 01:19:01 2013 -0800

----------------------------------------------------------------------
 .../org/apache/spark/deploy/DeployMessage.scala |  43 ++++-
 .../apache/spark/deploy/DriverDescription.scala |  27 +++
 .../apache/spark/deploy/client/AppClient.scala  |   2 +-
 .../spark/deploy/client/DriverClient.scala      | 129 ++++++++++++++
 .../apache/spark/deploy/master/DriverInfo.scala |  38 ++++
 .../spark/deploy/master/DriverState.scala       |  34 ++++
 .../master/FileSystemPersistenceEngine.scala    |  15 +-
 .../org/apache/spark/deploy/master/Master.scala | 152 ++++++++++++++--
 .../spark/deploy/master/PersistenceEngine.scala |  11 +-
 .../apache/spark/deploy/master/WorkerInfo.scala |  14 ++
 .../master/ZooKeeperPersistenceEngine.scala     |  14 +-
 .../spark/deploy/master/ui/IndexPage.scala      |  42 ++++-
 .../spark/deploy/worker/DriverRunner.scala      | 178 +++++++++++++++++++
 .../org/apache/spark/deploy/worker/Worker.scala |  60 +++++--
 .../spark/deploy/worker/ui/IndexPage.scala      |  36 +++-
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |  39 ++--
 16 files changed, 781 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 2753317..6743526 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -20,10 +20,11 @@ package org.apache.spark.deploy
 import scala.collection.immutable.List
 
 import org.apache.spark.deploy.ExecutorState.ExecutorState
-import org.apache.spark.deploy.master.{WorkerInfo, ApplicationInfo}
+import org.apache.spark.deploy.master.{DriverInfo, WorkerInfo, ApplicationInfo}
 import org.apache.spark.deploy.master.RecoveryState.MasterState
-import org.apache.spark.deploy.worker.ExecutorRunner
+import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
 import org.apache.spark.util.Utils
+import org.apache.spark.deploy.master.DriverState.DriverState
 
 
 private[deploy] sealed trait DeployMessage extends Serializable
@@ -54,7 +55,14 @@ private[deploy] object DeployMessages {
       exitStatus: Option[Int])
     extends DeployMessage
 
-  case class WorkerSchedulerStateResponse(id: String, executors: List[ExecutorDescription])
+  case class DriverStateChanged(
+      driverId: String,
+      state: DriverState,
+      exception: Option[Exception])
+    extends DeployMessage
+
+  case class WorkerSchedulerStateResponse(id: String, executors: List[ExecutorDescription],
+     driverIds: Seq[String])
 
   case class Heartbeat(workerId: String) extends DeployMessage
 
@@ -76,14 +84,19 @@ private[deploy] object DeployMessages {
       sparkHome: String)
     extends DeployMessage
 
-  // Client to Master
+  case class LaunchDriver(driverId: String, jarUrl: String, mainClass: String, mem: Int)
+    extends DeployMessage
+
+  case class KillDriver(driverId: String) extends DeployMessage
+
+  // AppClient to Master
 
   case class RegisterApplication(appDescription: ApplicationDescription)
     extends DeployMessage
 
   case class MasterChangeAcknowledged(appId: String)
 
-  // Master to Client
+  // Master to AppClient
 
   case class RegisteredApplication(appId: String, masterUrl: String) extends DeployMessage
 
@@ -97,11 +110,21 @@ private[deploy] object DeployMessages {
 
   case class ApplicationRemoved(message: String)
 
-  // Internal message in Client
+  // DriverClient <-> Master
+
+  case class RequestSubmitDriver(driverDescription: DriverDescription) extends DeployMessage
+
+  case class SubmitDriverResponse(success: Boolean, message: String) extends DeployMessage
+
+  case class RequestKillDriver(driverId: String) extends DeployMessage
+
+  case class KillDriverResponse(success: Boolean, message: String) extends DeployMessage
+
+  // Internal message in AppClient
 
-  case object StopClient
+  case object StopAppClient
 
-  // Master to Worker & Client
+  // Master to Worker & AppClient
 
   case class MasterChanged(masterUrl: String, masterWebUiUrl: String)
 
@@ -112,6 +135,7 @@ private[deploy] object DeployMessages {
   // Master to MasterWebUI
 
   case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo],
+    activeDrivers: Array[DriverInfo], completedDrivers: Array[DriverInfo],
     activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo],
     status: MasterState) {
 
@@ -128,7 +152,8 @@ private[deploy] object DeployMessages {
   // Worker to WorkerWebUI
 
   case class WorkerStateResponse(host: String, port: Int, workerId: String,
-    executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String,
+    executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner],
+    drivers: List[DriverRunner], finishedDrivers: List[DriverRunner], masterUrl: String,
     cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) {
 
     Utils.checkHost(host, "Required hostname")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
new file mode 100644
index 0000000..52f6b1b
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala
@@ -0,0 +1,27 @@
+/*
+ * 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
+
+private[spark] class DriverDescription(
+    val jarUrl: String,
+    val mainClass: String,
+    val mem: Integer) // TODO: Should this be Long?
+  extends Serializable {
+
+  override def toString: String = s"DriverDescription ($mainClass)"
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
index c5a0d1f..737ba09 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
@@ -147,7 +147,7 @@ private[spark] class AppClient(
         logWarning(s"Connection to $address failed; waiting for master to reconnect...")
         markDisconnected()
 
-      case StopClient =>
+      case StopAppClient =>
         markDead()
         sender ! true
         context.stop(self)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
new file mode 100644
index 0000000..482bafd
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -0,0 +1,129 @@
+/*
+ * 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.client
+
+import akka.actor._
+import akka.remote.{RemotingLifecycleEvent}
+
+import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.deploy.{DeployMessage, DriverDescription}
+import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.master.{MasterArguments, Master}
+import akka.pattern.ask
+
+import org.apache.spark.util.{Utils, AkkaUtils}
+import scala.concurrent.duration.{FiniteDuration, Duration}
+import java.util.concurrent.TimeUnit
+import akka.util.Timeout
+import scala.concurrent.Await
+import akka.actor.Actor.emptyBehavior
+
+/**
+ * Parent class for actors that to send a single message to the standalone master and then die.
+ */
+private[spark] abstract class SingleMessageClient(
+    actorSystem: ActorSystem, master: String, message: DeployMessage)
+  extends Logging {
+
+  // Concrete child classes must implement
+  def handleResponse(response: Any)
+
+  var actor: ActorRef = actorSystem.actorOf(Props(new DriverActor()))
+
+  class DriverActor extends Actor with Logging {
+    override def preStart() {
+      context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
+      logInfo("Sending message to master " + master + "...")
+      val masterActor = context.actorSelection(Master.toAkkaUrl(master))
+      val timeoutDuration: FiniteDuration = Duration.create(
+        System.getProperty("spark.akka.askTimeout", "10").toLong, TimeUnit.SECONDS)
+      val submitFuture = masterActor.ask(message)(timeoutDuration)
+      handleResponse(Await.result(submitFuture, timeoutDuration))
+      actorSystem.stop(actor)
+      actorSystem.shutdown()
+    }
+
+    override def receive = emptyBehavior
+  }
+}
+
+/**
+ * Submits a driver to the master.
+ */
+private[spark] class SubmissionClient(actorSystem: ActorSystem, master: String,
+    driverDescription: DriverDescription)
+    extends SingleMessageClient(actorSystem, master, RequestSubmitDriver(driverDescription)) {
+
+  override def handleResponse(response: Any) {
+    val resp = response.asInstanceOf[SubmitDriverResponse]
+    if (!resp.success) {
+      logError(s"Error submitting driver to $master")
+      logError(resp.message)
+    }
+  }
+}
+
+/**
+ * Terminates a client at the master.
+ */
+private[spark] class TerminationClient(actorSystem: ActorSystem, master: String, driverId: String)
+    extends SingleMessageClient(actorSystem, master, RequestKillDriver(driverId)) {
+
+  override def handleResponse(response: Any) {
+    val resp = response.asInstanceOf[KillDriverResponse]
+    if (!resp.success) {
+      logError(s"Error terminating $driverId at $master")
+      logError(resp.message)
+    }
+  }
+}
+
+/**
+ * Callable utility for starting and terminating drivers inside of the standalone scheduler.
+ */
+object DriverClient {
+
+  def main(args: Array[String]) {
+    if (args.size < 3) {
+      println("usage: DriverClient launch <active-master> <jar-url> <main-class>")
+      println("usage: DriverClient kill <active-master> <driver-id>")
+      System.exit(-1)
+    }
+
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(
+      "driverSubmission", Utils.localHostName(), 0)
+
+    // TODO Should be configurable
+    val mem = 512
+
+    args(0) match {
+      case "launch" =>
+        val master = args(1)
+        val jarUrl = args(2)
+        val mainClass = args(3)
+        val driverDescription = new DriverDescription(jarUrl, mainClass, mem)
+        val client = new SubmissionClient(actorSystem, master, driverDescription)
+
+      case "kill" =>
+        val master = args(1)
+        val driverId = args(2)
+        val client = new TerminationClient(actorSystem, master, driverId)
+    }
+    actorSystem.awaitTermination()
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
new file mode 100644
index 0000000..69d150a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
@@ -0,0 +1,38 @@
+/*
+ * 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.master
+
+import org.apache.spark.deploy.{DriverDescription, ApplicationDescription}
+import java.util.Date
+import akka.actor.ActorRef
+import scala.collection.mutable
+
+private[spark] class DriverInfo(
+    val startTime: Long,
+    val id: String,
+    val desc: DriverDescription,
+    val submitDate: Date)
+  extends Serializable {
+
+  @transient var state: DriverState.Value = DriverState.SUBMITTED
+  /* If we fail when launching the driver, the exception is stored here. */
+  @transient var exception: Option[Exception] = None
+  /* Most recent worker assigned to this driver */
+  @transient var worker: Option[WorkerInfo] = None
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
new file mode 100644
index 0000000..230dab1
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
@@ -0,0 +1,34 @@
+/*
+ * 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.master
+
+private[spark] object DriverState extends Enumeration {
+
+  type DriverState = Value
+
+  // SUBMITTED: Submitted but not yet scheduled on a worker
+  // RUNNING: Has been allocated to a worker to run
+  // FINISHED: Previously ran and exited cleanly
+  // RELAUNCHING: Exited non-zero or due to worker failure, but has not yet started running again
+  // UNKNOWN: The state of the driver is temporarily not known due to master failure recovery
+  // KILLED: A user manually killed this driver
+  // FAILED: Unable to run due to an unrecoverable error (e.g. missing jar file)
+  val SUBMITTED, RUNNING, FINISHED, RELAUNCHING, UNKNOWN, KILLED, FAILED = Value
+
+  val MAX_NUM_RETRY = 10
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
index 043945a..44a046b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
@@ -47,6 +47,15 @@ private[spark] class FileSystemPersistenceEngine(
     new File(dir + File.separator + "app_" + app.id).delete()
   }
 
+  override def addDriver(driver: DriverInfo) {
+    val driverFile = new File(dir + File.separator + "driver_" + driver.id)
+    serializeIntoFile(driverFile, driver)
+  }
+
+  override def removeDriver(driver: DriverInfo) {
+    new File(dir + File.separator + "driver_" + driver.id).delete()
+  }
+
   override def addWorker(worker: WorkerInfo) {
     val workerFile = new File(dir + File.separator + "worker_" + worker.id)
     serializeIntoFile(workerFile, worker)
@@ -56,13 +65,15 @@ private[spark] class FileSystemPersistenceEngine(
     new File(dir + File.separator + "worker_" + worker.id).delete()
   }
 
-  override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
+  override def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = {
     val sortedFiles = new File(dir).listFiles().sortBy(_.getName)
     val appFiles = sortedFiles.filter(_.getName.startsWith("app_"))
     val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
+    val driverFiles = sortedFiles.filter(_.getName.startsWith("driver_"))
+    val drivers = driverFiles.map(deserializeFromFile[DriverInfo])
     val workerFiles = sortedFiles.filter(_.getName.startsWith("worker_"))
     val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
-    (apps, workers)
+    (apps, drivers, workers)
   }
 
   private def serializeIntoFile(file: File, value: AnyRef) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index eebd079..76af332 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -30,7 +30,7 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
 import akka.serialization.SerializationExtension
 
 import org.apache.spark.{Logging, SparkException}
-import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
+import org.apache.spark.deploy.{DriverDescription, ApplicationDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.MasterMessages._
 import org.apache.spark.deploy.master.ui.MasterWebUI
@@ -47,7 +47,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
   val RECOVERY_DIR = System.getProperty("spark.deploy.recoveryDirectory", "")
   val RECOVERY_MODE = System.getProperty("spark.deploy.recoveryMode", "NONE")
 
-  var nextAppNumber = 0
   val workers = new HashSet[WorkerInfo]
   val idToWorker = new HashMap[String, WorkerInfo]
   val actorToWorker = new HashMap[ActorRef, WorkerInfo]
@@ -57,9 +56,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
   val idToApp = new HashMap[String, ApplicationInfo]
   val actorToApp = new HashMap[ActorRef, ApplicationInfo]
   val addressToApp = new HashMap[Address, ApplicationInfo]
-
   val waitingApps = new ArrayBuffer[ApplicationInfo]
   val completedApps = new ArrayBuffer[ApplicationInfo]
+  var nextAppNumber = 0
+
+  val drivers = new HashSet[DriverInfo]
+  val completedDrivers = new ArrayBuffer[DriverInfo]
+  val waitingDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling
+  var nextDriverNumber = 0
 
   Utils.checkHost(host, "Expected hostname")
 
@@ -134,14 +138,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
 
   override def receive = {
     case ElectedLeader => {
-      val (storedApps, storedWorkers) = persistenceEngine.readPersistedData()
-      state = if (storedApps.isEmpty && storedWorkers.isEmpty)
+      val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData()
+      state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty)
         RecoveryState.ALIVE
       else
         RecoveryState.RECOVERING
       logInfo("I have been elected leader! New state: " + state)
       if (state == RecoveryState.RECOVERING) {
-        beginRecovery(storedApps, storedWorkers)
+        beginRecovery(storedApps, storedDrivers, storedWorkers)
         context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() }
       }
     }
@@ -168,6 +172,52 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       }
     }
 
+    case RequestSubmitDriver(description) => {
+      if (state == RecoveryState.STANDBY) {
+        sender ! SubmitDriverResponse(false, "Standby master cannot accept driver submission")
+      } else {
+        logInfo("Driver submitted " + description.mainClass)
+        val driver = createDriver(description)
+        persistenceEngine.addDriver(driver)
+        waitingDrivers += driver
+        drivers.add(driver)
+        schedule()
+
+        // TODO: It might be good to instead have the submission client poll the master to determine
+        //       the current status of the driver. Since we may already want to expose this.
+
+        sender ! SubmitDriverResponse(true, "Driver successfully submitted")
+      }
+    }
+
+    case RequestKillDriver(driverId) => {
+      if (state == RecoveryState.STANDBY) {
+        sender ! KillDriverResponse(false, "Standby master cannot kill drivers")
+      } else {
+        logInfo("Asked to kill driver " + driverId)
+        val driver = drivers.find(_.id == driverId)
+        driver match {
+          case Some(d) =>
+            if (waitingDrivers.contains(d)) { waitingDrivers -= d }
+            else {
+              // We just notify the worker to kill the driver here. The final bookkeeping occurs
+              // on the return path when the worker submits a state change back to the master
+              // to notify it that the driver was successfully killed.
+              d.worker.foreach { w =>
+                w.actor ! KillDriver(driverId)
+              }
+            }
+            val msg = s"Kill request for $driverId submitted"
+            logInfo(msg)
+            sender ! KillDriverResponse(true, msg)
+          case None =>
+            val msg = s"Could not find running driver $driverId"
+            logWarning(msg)
+            sender ! KillDriverResponse(false, msg)
+        }
+      }
+    }
+
     case RegisterApplication(description) => {
       if (state == RecoveryState.STANDBY) {
         // ignore, don't send response
@@ -210,6 +260,25 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       }
     }
 
+    case DriverStateChanged(driverId, state, exception) => {
+      if (!(state == DriverState.FAILED || state == DriverState.FINISHED ||
+          state == DriverState.KILLED)) {
+        throw new Exception(s"Received unexpected state update for driver $driverId: $state")
+      }
+      drivers.find(_.id == driverId) match {
+        case Some(driver) => {
+          drivers -= driver
+          completedDrivers += driver
+          persistenceEngine.removeDriver(driver)
+          driver.state = state
+          driver.exception = exception
+          driver.worker.foreach(w => w.removeDriver(driver))
+        }
+        case None =>
+          logWarning(s"Got driver update for unknown driver $driverId")
+      }
+    }
+
     case Heartbeat(workerId) => {
       idToWorker.get(workerId) match {
         case Some(workerInfo) =>
@@ -231,7 +300,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       if (canCompleteRecovery) { completeRecovery() }
     }
 
-    case WorkerSchedulerStateResponse(workerId, executors) => {
+    case WorkerSchedulerStateResponse(workerId, executors, driverIds) => {
       idToWorker.get(workerId) match {
         case Some(worker) =>
           logInfo("Worker has been re-registered: " + workerId)
@@ -244,6 +313,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
             worker.addExecutor(execInfo)
             execInfo.copyState(exec)
           }
+
+          for (driverId <- driverIds) {
+            drivers.find(_.id == driverId).foreach { driver =>
+              driver.worker = Some(worker)
+              driver.state = DriverState.RUNNING
+              worker.drivers(driverId) = driver
+            }
+          }
         case None =>
           logWarning("Scheduler state from unknown worker: " + workerId)
       }
@@ -260,8 +337,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }
 
     case RequestMasterState => {
-      sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray,
-        state)
+      sender ! MasterStateResponse(host, port, workers.toArray, drivers.toArray,
+        completedDrivers.toArray ,apps.toArray, completedApps.toArray, state)
     }
 
     case CheckForWorkerTimeOut => {
@@ -277,7 +354,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     workers.count(_.state == WorkerState.UNKNOWN) == 0 &&
       apps.count(_.state == ApplicationState.UNKNOWN) == 0
 
-  def beginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo]) {
+  def beginRecovery(storedApps: Seq[ApplicationInfo], storedDrivers: Seq[DriverInfo],
+      storedWorkers: Seq[WorkerInfo]) {
     for (app <- storedApps) {
       logInfo("Trying to recover app: " + app.id)
       try {
@@ -289,6 +367,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       }
     }
 
+    for (driver <- storedDrivers) {
+      // Here we just read in the list of drivers. Any drivers associated with now-lost workers
+      // will be re-launched when we detect that the worker is missing.
+      drivers += driver
+    }
+
     for (worker <- storedWorkers) {
       logInfo("Trying to recover worker: " + worker.id)
       try {
@@ -312,6 +396,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     workers.filter(_.state == WorkerState.UNKNOWN).foreach(removeWorker)
     apps.filter(_.state == ApplicationState.UNKNOWN).foreach(finishApplication)
 
+    // Reschedule drivers which were not claimed by any workers
+    drivers.filter(_.worker.isEmpty).foreach { d =>
+      logWarning(s"Driver ${d.id} was not found after master recovery, re-launching")
+      relaunchDriver(d)
+    }
+
     state = RecoveryState.ALIVE
     schedule()
     logInfo("Recovery complete - resuming operations!")
@@ -332,6 +422,16 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
    */
   def schedule() {
     if (state != RecoveryState.ALIVE) { return }
+    // First schedule drivers, they take strict precedence over applications
+    for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) {
+      for (driver <- Seq(waitingDrivers: _*)) {
+        if (worker.memoryFree > driver.desc.mem) {
+          launchDriver(worker, driver)
+          waitingDrivers -= driver
+        }
+      }
+    }
+
     // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app
     // in the queue, then the second app, etc.
     if (spreadOutApps) {
@@ -418,9 +518,19 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         exec.id, ExecutorState.LOST, Some("worker lost"), None)
       exec.application.removeExecutor(exec)
     }
+    for (driver <- worker.drivers.values) {
+      relaunchDriver(driver)
+    }
     persistenceEngine.removeWorker(worker)
   }
 
+  def relaunchDriver(driver: DriverInfo) {
+    driver.worker = None
+    driver.state = DriverState.RELAUNCHING
+    waitingDrivers += driver
+    schedule()
+  }
+
   def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
     val now = System.currentTimeMillis()
     val date = new Date(now)
@@ -499,6 +609,28 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       }
     }
   }
+
+  /** Generate a new driver ID given a driver's submission date */
+  def newDriverId(submitDate: Date): String = {
+    val appId = "driver-%s-%04d".format(DATE_FORMAT.format(submitDate), nextDriverNumber)
+    nextDriverNumber += 1
+    appId
+  }
+
+  def createDriver(desc: DriverDescription): DriverInfo = {
+    val now = System.currentTimeMillis()
+    val date = new Date(now)
+    new DriverInfo(now, newDriverId(date), desc, date)
+  }
+
+  def launchDriver(worker: WorkerInfo, driver: DriverInfo) {
+    logInfo("Launching driver " + driver.id + " on worker " + worker.id)
+    worker.addDriver(driver)
+    driver.worker = Some(worker)
+    worker.actor ! LaunchDriver(driver.id, driver.desc.jarUrl, driver.desc.mainClass,
+      driver.desc.mem)
+    driver.state = DriverState.RUNNING
+  }
 }
 
 private[spark] object Master {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
index 94b986c..e3640ea 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
@@ -35,11 +35,15 @@ private[spark] trait PersistenceEngine {
 
   def removeWorker(worker: WorkerInfo)
 
+  def addDriver(driver: DriverInfo)
+
+  def removeDriver(driver: DriverInfo)
+
   /**
    * Returns the persisted data sorted by their respective ids (which implies that they're
    * sorted by time of creation).
    */
-  def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo])
+  def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo])
 
   def close() {}
 }
@@ -49,5 +53,8 @@ private[spark] class BlackHolePersistenceEngine extends PersistenceEngine {
   override def removeApplication(app: ApplicationInfo) {}
   override def addWorker(worker: WorkerInfo) {}
   override def removeWorker(worker: WorkerInfo) {}
-  override def readPersistedData() = (Nil, Nil)
+  override def addDriver(driver: DriverInfo) {}
+  override def removeDriver(driver: DriverInfo) {}
+
+  override def readPersistedData() = (Nil, Nil, Nil)
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index e05f587..27c2ff4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -36,6 +36,7 @@ private[spark] class WorkerInfo(
   assert (port > 0)
 
   @transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // fullId => info
+  @transient var drivers: mutable.HashMap[String, DriverInfo] = _
   @transient var state: WorkerState.Value = _
   @transient var coresUsed: Int = _
   @transient var memoryUsed: Int = _
@@ -54,6 +55,7 @@ private[spark] class WorkerInfo(
 
   private def init() {
     executors = new mutable.HashMap
+    drivers = new mutable.HashMap
     state = WorkerState.ALIVE
     coresUsed = 0
     memoryUsed = 0
@@ -83,6 +85,18 @@ private[spark] class WorkerInfo(
     executors.values.exists(_.application == app)
   }
 
+  def addDriver(driver: DriverInfo) {
+    drivers(driver.id) = driver
+    memoryUsed += driver.desc.mem
+    coresUsed += 1
+  }
+
+  def removeDriver(driver: DriverInfo) {
+    drivers -= driver.id
+    memoryUsed -= driver.desc.mem
+    coresUsed -= 1
+  }
+
   def webUiAddress : String = {
     "http://" + this.publicAddress + ":" + this.webUiPort
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
index 825344b..52df173 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -49,6 +49,14 @@ class ZooKeeperPersistenceEngine(serialization: Serialization)
     zk.delete(WORKING_DIR + "/app_" + app.id)
   }
 
+  override def addDriver(driver: DriverInfo) {
+    serializeIntoFile(WORKING_DIR + "/driver_" + driver.id, driver)
+  }
+
+  override def removeDriver(driver: DriverInfo) {
+    zk.delete(WORKING_DIR + "/driver_" + driver.id)
+  }
+
   override def addWorker(worker: WorkerInfo) {
     serializeIntoFile(WORKING_DIR + "/worker_" + worker.id, worker)
   }
@@ -61,13 +69,15 @@ class ZooKeeperPersistenceEngine(serialization: Serialization)
     zk.close()
   }
 
-  override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
+  override def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = {
     val sortedFiles = zk.getChildren(WORKING_DIR).toList.sorted
     val appFiles = sortedFiles.filter(_.startsWith("app_"))
     val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
+    val driverFiles = sortedFiles.filter(_.startsWith("driver_"))
+    val drivers = driverFiles.map(deserializeFromFile[DriverInfo])
     val workerFiles = sortedFiles.filter(_.startsWith("worker_"))
     val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
-    (apps, workers)
+    (apps, drivers, workers)
   }
 
   private def serializeIntoFile(path: String, value: AnyRef) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index 4ef7628..13903b4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -26,7 +26,8 @@ import net.liftweb.json.JsonAST.JValue
 
 import org.apache.spark.deploy.{DeployWebUI, JsonProtocol}
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
-import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo}
+import org.apache.spark.deploy.JsonProtocol
+import org.apache.spark.deploy.master.{DriverInfo, ApplicationInfo, WorkerInfo}
 import org.apache.spark.ui.UIUtils
 import org.apache.spark.util.Utils
 
@@ -56,6 +57,12 @@ private[spark] class IndexPage(parent: MasterWebUI) {
     val completedApps = state.completedApps.sortBy(_.endTime).reverse
     val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps)
 
+    val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Memory", "Main Class")
+    val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse
+    val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers)
+    val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse
+    val completedDriversTable = UIUtils.listingTable(driverHeaders, driverRow, completedDrivers)
+
     val content =
         <div class="row-fluid">
           <div class="span12">
@@ -70,6 +77,9 @@ private[spark] class IndexPage(parent: MasterWebUI) {
               <li><strong>Applications:</strong>
                 {state.activeApps.size} Running,
                 {state.completedApps.size} Completed </li>
+              <li><strong>Drivers:</strong>
+                {state.activeDrivers.size} Running,
+                {state.completedDrivers.size} Completed </li>
             </ul>
           </div>
         </div>
@@ -94,7 +104,22 @@ private[spark] class IndexPage(parent: MasterWebUI) {
             <h4> Completed Applications </h4>
             {completedAppsTable}
           </div>
-        </div>;
+        </div>
+
+          <div class="row-fluid">
+            <div class="span12">
+              <h4> Active Drivers </h4>
+
+              {activeDriversTable}
+            </div>
+          </div>
+
+          <div class="row-fluid">
+            <div class="span12">
+              <h4> Completed Drivers </h4>
+              {completedDriversTable}
+            </div>
+          </div>;
     UIUtils.basicSparkPage(content, "Spark Master at " + state.uri)
   }
 
@@ -134,4 +159,17 @@ private[spark] class IndexPage(parent: MasterWebUI) {
       <td>{DeployWebUI.formatDuration(app.duration)}</td>
     </tr>
   }
+
+  def driverRow(driver: DriverInfo): Seq[Node] = {
+    <tr>
+      <td>{driver.id} </td>
+      <td>{driver.submitDate}</td>
+      <td>{driver.worker.map(w => w.id.toString).getOrElse("None")}</td>
+      <td>{driver.state}</td>
+      <td sorttable_customkey={driver.desc.mem.toString}>
+        {Utils.megabytesToString(driver.desc.mem.toLong)}
+      </td>
+      <td>{driver.desc.mainClass}</td>
+    </tr>
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
new file mode 100644
index 0000000..fccc36b
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -0,0 +1,178 @@
+/*
+ * 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.worker
+
+import java.io._
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+
+import org.apache.spark.Logging
+import org.apache.spark.util.Utils
+import org.apache.hadoop.fs.{FileUtil, Path}
+import org.apache.hadoop.conf.Configuration
+import akka.actor.{ActorRef, ActorSelection}
+import org.apache.spark.deploy.DeployMessages.DriverStateChanged
+import org.apache.spark.deploy.master.DriverState
+
+/**
+ * Manages the execution of one driver process.
+ */
+private[spark] class DriverRunner(
+    val driverId: String,
+    val jarUrl: String,
+    val mainClass: String,
+    val workDir: File,
+    val memory: Int,
+    val worker: ActorRef)
+  extends Logging {
+
+  var process: Option[Process] = None
+  @volatile var killed = false
+
+  /** Starts a thread to run and manage the driver. */
+  def start() = {
+    new Thread("DriverRunner for " + driverId) {
+      override def run() {
+        var exn: Option[Exception] = None
+
+        try {
+          val driverDir = createWorkingDirectory()
+          val localJarFilename = downloadUserJar(driverDir)
+          val command = Seq("java", "-cp", localJarFilename, mainClass)
+          runCommandWithRetry(command, driverDir)
+        }
+        catch {
+          case e: Exception => exn = Some(e)
+        }
+
+        val finalState =
+          if (killed) { DriverState.KILLED }
+          else if (exn.isDefined) { DriverState.FAILED }
+          else { DriverState.FINISHED }
+
+        worker ! DriverStateChanged(driverId, finalState, exn)
+      }
+    }.start()
+  }
+
+  /** Terminate this driver (or prevent it from ever starting if not yet started) */
+  def kill() {
+    killed = true
+    process.foreach(p => p.destroy())
+  }
+
+  /** Spawn a thread that will redirect a given stream to a file */
+  def redirectStream(in: InputStream, file: File) {
+    val out = new FileOutputStream(file, true)
+    new Thread("redirect output to " + file) {
+      override def run() {
+        try {
+          Utils.copyStream(in, out, true)
+        } catch {
+          case e: IOException =>
+            logInfo("Redirection to " + file + " closed: " + e.getMessage)
+        }
+      }
+    }.start()
+  }
+
+  /**
+   * Creates the working directory for this driver.
+   * Will throw an exception if there are errors preparing the directory.
+   */
+  def createWorkingDirectory(): File = {
+    val driverDir = new File(workDir, driverId)
+    if (!driverDir.exists() && !driverDir.mkdirs()) {
+      throw new IOException("Failed to create directory " + driverDir)
+    }
+    driverDir
+  }
+
+  /**
+   * Download the user jar into the supplied directory and return its local path.
+   * Will throw an exception if there are errors downloading the jar.
+   */
+  def downloadUserJar(driverDir: File): String = {
+
+    val jarPath = new Path(jarUrl)
+
+    val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path
+    val jarFileSystem = jarPath.getFileSystem(emptyConf)
+
+    val destPath = new Path(driverDir.getAbsolutePath())
+    val destFileSystem = destPath.getFileSystem(emptyConf)
+    val jarFileName = jarPath.getName
+    val localJarFile = new File(driverDir, jarFileName)
+    val localJarFilename = localJarFile.getAbsolutePath
+
+    if (!localJarFile.exists()) { // May already exist if running multiple workers on one node
+      logInfo(s"Copying user jar $jarPath to $destPath")
+      FileUtil.copy(jarFileSystem, jarPath, destFileSystem, destPath, false, false, emptyConf)
+    }
+
+    if (!localJarFile.exists()) { // Verify copy succeeded
+      throw new Exception(s"Did not see expected jar $jarFileName in $driverDir")
+    }
+
+    localJarFilename
+  }
+
+  /** Continue launching the supplied command until it exits zero. */
+  def runCommandWithRetry(command: Seq[String], baseDir: File) = {
+    /* Time to wait between submission retries. */
+    var waitSeconds = 1
+    // TODO: We should distinguish between "immediate" exits and cases where it was running
+    //       for a long time and then exits.
+    var cleanExit = false
+
+    while (!cleanExit && !killed) {
+      Thread.sleep(waitSeconds * 1000)
+      val builder = new ProcessBuilder(command: _*).directory(baseDir)
+      logInfo("Launch command: " + command.mkString("\"", "\" \"", "\""))
+
+      process = Some(builder.start())
+
+      // Redirect stdout and stderr to files
+      val stdout = new File(baseDir, "stdout")
+      redirectStream(process.get.getInputStream, stdout)
+
+      val stderr = new File(baseDir, "stderr")
+      val header = "Driver Command: %s\n%s\n\n".format(
+        command.mkString("\"", "\" \"", "\""), "=" * 40)
+      Files.write(header, stderr, Charsets.UTF_8)
+      redirectStream(process.get.getErrorStream, stderr)
+
+
+      val exitCode =
+        /* There is a race here I've elected to ignore for now because it's very unlikely and not
+         * simple to fix. This could see `killed=false` then the main thread gets a kill request
+         * and sets `killed=true` and destroys the not-yet-started process, then this thread
+         * launches the process. For now, in that case the user can just re-submit the kill
+         * request. */
+        if (killed) -1
+        else process.get.waitFor()
+
+      cleanExit = exitCode == 0
+      if (!cleanExit && !killed) {
+        waitSeconds = waitSeconds * 2 // exponential back-off
+        logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.")
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 87531b6..a2b491a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -30,18 +30,10 @@ import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent}
 import org.apache.spark.{SparkException, Logging}
 import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
-import org.apache.spark.deploy.master.Master
+import org.apache.spark.deploy.master.{DriverState, Master}
 import org.apache.spark.deploy.worker.ui.WorkerWebUI
 import org.apache.spark.metrics.MetricsSystem
 import org.apache.spark.util.{Utils, AkkaUtils}
-import org.apache.spark.deploy.DeployMessages.WorkerStateResponse
-import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed
-import org.apache.spark.deploy.DeployMessages.KillExecutor
-import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged
-import org.apache.spark.deploy.DeployMessages.Heartbeat
-import org.apache.spark.deploy.DeployMessages.RegisteredWorker
-import org.apache.spark.deploy.DeployMessages.LaunchExecutor
-import org.apache.spark.deploy.DeployMessages.RegisterWorker
 
 /**
   * @param masterUrls Each url should look like spark://host:port.
@@ -83,6 +75,9 @@ private[spark] class Worker(
   var workDir: File = null
   val executors = new HashMap[String, ExecutorRunner]
   val finishedExecutors = new HashMap[String, ExecutorRunner]
+  val drivers = new HashMap[String, DriverRunner]
+  val finishedDrivers = new HashMap[String, DriverRunner]
+
   val publicAddress = {
     val envVar = System.getenv("SPARK_PUBLIC_DNS")
     if (envVar != null) envVar else host
@@ -193,7 +188,7 @@ private[spark] class Worker(
 
       val execs = executors.values.
         map(e => new ExecutorDescription(e.appId, e.execId, e.cores, e.state))
-      sender ! WorkerSchedulerStateResponse(workerId, execs.toList)
+      sender ! WorkerSchedulerStateResponse(workerId, execs.toList, drivers.keys.toSeq)
 
     case RegisterWorkerFailed(message) =>
       if (!registered) {
@@ -247,13 +242,56 @@ private[spark] class Worker(
         }
       }
 
+    case LaunchDriver(driverId, jarUrl, mainClass, memory) => {
+      logInfo(s"Asked to launch driver $driverId")
+      val driver = new DriverRunner(driverId, jarUrl, mainClass, workDir, memory, self)
+      drivers(driverId) = driver
+      driver.start()
+
+      coresUsed += 1
+      memoryUsed += memory
+    }
+
+    case KillDriver(driverId) => {
+      logInfo(s"Asked to kill driver $driverId")
+
+      drivers.find(_._1 == driverId) match {
+        case Some((id, runner)) =>
+          runner.kill()
+        case None =>
+          logError(s"Asked to kill unknown driver $driverId")
+      }
+
+    }
+
+
+    case DriverStateChanged(driverId, state, exception) => {
+      state match {
+        case DriverState.FAILED =>
+          logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}")
+        case DriverState.FINISHED =>
+          logInfo(s"Driver $driverId exited successfully")
+        case DriverState.KILLED =>
+          logInfo(s"Driver $driverId was killed")
+      }
+      masterLock.synchronized {
+        master ! DriverStateChanged(driverId, state, exception)
+      }
+      val driver = drivers(driverId)
+      memoryUsed -= driver.memory
+      coresUsed -= 1
+      drivers -= driverId
+      finishedDrivers(driverId) = driver
+    }
+
     case x: DisassociatedEvent if x.remoteAddress == masterAddress =>
       logInfo(s"$x Disassociated !")
       masterDisconnected()
 
     case RequestWorkerState => {
       sender ! WorkerStateResponse(host, port, workerId, executors.values.toList,
-        finishedExecutors.values.toList, activeMasterUrl, cores, memory,
+        finishedExecutors.values.toList, drivers.values.toList,
+        finishedDrivers.values.toList, activeMasterUrl, cores, memory,
         coresUsed, memoryUsed, activeMasterWebUiUrl)
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index 0d59048..e233b82 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -30,7 +30,7 @@ import net.liftweb.json.JsonAST.JValue
 
 import org.apache.spark.deploy.JsonProtocol
 import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse}
-import org.apache.spark.deploy.worker.ExecutorRunner
+import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
 import org.apache.spark.ui.UIUtils
 import org.apache.spark.util.Utils
 
@@ -56,6 +56,12 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
     val finishedExecutorTable =
       UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors)
 
+    val driverHeaders = Seq("DriverID", "Main Class", "Memory", "Logs")
+    val runningDriverTable =
+      UIUtils.listingTable(driverHeaders, driverRow, workerState.drivers)
+    def finishedDriverTable =
+      UIUtils.listingTable(driverHeaders, driverRow, workerState.finishedDrivers)
+
     val content =
         <div class="row-fluid"> <!-- Worker Details -->
           <div class="span12">
@@ -84,6 +90,20 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
             <h4> Finished Executors </h4>
             {finishedExecutorTable}
           </div>
+        </div>
+
+        <div class="row-fluid"> <!-- Running Drivers -->
+          <div class="span12">
+            <h4> Running Drivers {workerState.drivers.size} </h4>
+            {runningDriverTable}
+          </div>
+        </div>
+
+        <div class="row-fluid"> <!-- Finished Drivers  -->
+          <div class="span12">
+            <h4> Finished Drivers </h4>
+            {finishedDriverTable}
+          </div>
         </div>;
 
     UIUtils.basicSparkPage(content, "Spark Worker at %s:%s".format(
@@ -111,6 +131,20 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
           .format(executor.appId, executor.execId)}>stderr</a>
       </td> 
     </tr>
+
   }
 
+  def driverRow(driver: DriverRunner): Seq[Node] = {
+    <tr>
+      <td>{driver.driverId}</td>
+      <td>{driver.mainClass}</td>
+      <td sorttable_customkey={driver.memory.toString}>
+        {Utils.megabytesToString(driver.memory)}
+      </td>
+      <td>
+        <a href={s"logPage?driverId=${driver.driverId}&logType=stdout"}>stdout</a>
+        <a href={s"logPage?driverId=${driver.driverId}&logType=stderr"}>stderr</a>
+      </td>
+    </tr>
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a4acc4c/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index 40d6bdb..d128e58 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -69,30 +69,44 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
 
   def log(request: HttpServletRequest): String = {
     val defaultBytes = 100 * 1024
-    val appId = request.getParameter("appId")
-    val executorId = request.getParameter("executorId")
+
+    val appId = Option(request.getParameter("appId"))
+    val executorId = Option(request.getParameter("executorId"))
+    val driverId = Option(request.getParameter("driverId"))
     val logType = request.getParameter("logType")
     val offset = Option(request.getParameter("offset")).map(_.toLong)
     val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes)
-    val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType)
+
+    val path = (appId, executorId, driverId) match {
+      case (Some(a), Some(e), None) =>
+        s"${workDir.getPath}/$appId/$executorId/$logType"
+      case (None, None, Some(d)) =>
+        s"${workDir.getPath}/$driverId/$logType"
+    }
 
     val (startByte, endByte) = getByteRange(path, offset, byteLength)
     val file = new File(path)
     val logLength = file.length
 
-    val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n"
-      .format(startByte, endByte, logLength, appId, executorId, logType)
+    val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n"
     pre + Utils.offsetBytes(path, startByte, endByte)
   }
 
   def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = {
     val defaultBytes = 100 * 1024
-    val appId = request.getParameter("appId")
-    val executorId = request.getParameter("executorId")
+    val appId = Option(request.getParameter("appId"))
+    val executorId = Option(request.getParameter("executorId"))
+    val driverId = Option(request.getParameter("driverId"))
     val logType = request.getParameter("logType")
     val offset = Option(request.getParameter("offset")).map(_.toLong)
     val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes)
-    val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType)
+
+    val (path, params) = (appId, executorId, driverId) match {
+      case (Some(a), Some(e), None) =>
+        (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e")
+      case (None, None, Some(d)) =>
+        (s"${workDir.getPath}/$d/$logType", s"driverId=$d")
+    }
 
     val (startByte, endByte) = getByteRange(path, offset, byteLength)
     val file = new File(path)
@@ -106,9 +120,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
 
     val backButton =
       if (startByte > 0) {
-        <a href={"?appId=%s&executorId=%s&logType=%s&offset=%s&byteLength=%s"
-          .format(appId, executorId, logType, math.max(startByte-byteLength, 0),
-          byteLength)}>
+        <a href={"?%s&logType=%s&offset=%s&byteLength=%s"
+          .format(params, logType, math.max(startByte-byteLength, 0), byteLength)}>
           <button type="button" class="btn btn-default">
             Previous {Utils.bytesToString(math.min(byteLength, startByte))}
           </button>
@@ -122,8 +135,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
 
     val nextButton =
       if (endByte < logLength) {
-        <a href={"?appId=%s&executorId=%s&logType=%s&offset=%s&byteLength=%s".
-          format(appId, executorId, logType, endByte, byteLength)}>
+        <a href={"?%s&logType=%s&offset=%s&byteLength=%s".
+          format(params, logType, endByte, byteLength)}>
           <button type="button" class="btn btn-default">
             Next {Utils.bytesToString(math.min(byteLength, logLength-endByte))}
           </button>


[33/37] git commit: Merge remote-tracking branch 'apache-github/master' into standalone-driver

Posted by pw...@apache.org.
Merge remote-tracking branch 'apache-github/master' into standalone-driver

Conflicts:
	core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
	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/bc81ce04
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/bc81ce04
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/bc81ce04

Branch: refs/heads/master
Commit: bc81ce040ded3fe791ecffb15baa408346afc92a
Parents: 3ec21f2 bb6a39a
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Jan 8 00:38:31 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Jan 8 00:38:31 2014 -0800

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 README.md                                       |  20 +-
 assembly/pom.xml                                |   6 +-
 bin/pyspark                                     |   2 +-
 bin/run-example                                 |   9 +-
 bin/spark-class                                 |   2 +-
 conf/spark-env.sh.template                      |   2 +-
 .../apache/spark/network/netty/FileClient.java  |   5 +-
 .../netty/FileClientChannelInitializer.java     |   2 +-
 .../apache/spark/network/netty/FileServer.java  |   8 +-
 .../netty/FileServerChannelInitializer.java     |   4 +-
 .../spark/network/netty/FileServerHandler.java  |   6 +-
 .../spark/network/netty/PathResolver.java       |  52 +--
 .../org/apache/spark/default-log4j.properties   |   8 -
 .../org/apache/spark/log4j-defaults.properties  |   8 +
 .../main/scala/org/apache/spark/Logging.scala   |  10 +-
 .../org/apache/spark/MapOutputTracker.scala     |  14 +-
 .../main/scala/org/apache/spark/SparkConf.scala |  10 +-
 .../scala/org/apache/spark/SparkContext.scala   |   4 +
 .../main/scala/org/apache/spark/SparkEnv.scala  |  16 +-
 .../spark/deploy/ApplicationDescription.scala   |   2 +-
 .../apache/spark/deploy/client/TestClient.scala |   2 +-
 .../spark/deploy/master/ApplicationInfo.scala   |   7 +-
 .../org/apache/spark/deploy/master/Master.scala |  11 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |   2 +-
 .../spark/storage/BlockManagerMaster.scala      |   8 +-
 .../apache/spark/storage/ThreadingTest.scala    |   2 +-
 .../scala/org/apache/spark/util/AkkaUtils.scala |  13 +-
 .../apache/spark/MapOutputTrackerSuite.scala    |  17 +-
 .../apache/spark/deploy/JsonProtocolSuite.scala |   8 +-
 .../deploy/worker/ExecutorRunnerTest.scala      |   2 +-
 .../spark/storage/BlockManagerSuite.scala       |  12 +-
 .../spark/storage/DiskBlockManagerSuite.scala   |   4 +-
 docs/README.md                                  |   4 +-
 docs/_plugins/copy_api_dirs.rb                  |   4 +-
 docs/api.md                                     |   2 +-
 docs/configuration.md                           |  46 ++-
 docs/css/bootstrap.min.css                      |   2 +-
 docs/hadoop-third-party-distributions.md        |   2 +-
 docs/index.md                                   |   6 +-
 docs/job-scheduling.md                          |   5 +-
 docs/mllib-guide.md                             | 330 +++++++++++++++----
 docs/python-programming-guide.md                |   2 +-
 docs/quick-start.md                             |   8 +-
 docs/running-on-yarn.md                         |   6 +-
 docs/scala-programming-guide.md                 |   2 +-
 docs/spark-standalone.md                        |  10 +
 examples/pom.xml                                |  45 ++-
 .../org/apache/spark/examples/JavaHdfsLR.java   |  29 +-
 .../org/apache/spark/examples/JavaKMeans.java   |  23 +-
 .../org/apache/spark/examples/JavaLogQuery.java |  20 +-
 .../org/apache/spark/examples/JavaPageRank.java |  12 +-
 .../org/apache/spark/examples/JavaSparkPi.java  |  10 +-
 .../java/org/apache/spark/examples/JavaTC.java  |  16 +-
 .../apache/spark/examples/JavaWordCount.java    |  10 +-
 .../apache/spark/mllib/examples/JavaALS.java    |  18 +-
 .../apache/spark/mllib/examples/JavaKMeans.java |  16 +-
 .../org/apache/spark/mllib/examples/JavaLR.java |  19 +-
 .../streaming/examples/JavaFlumeEventCount.java |  15 +-
 .../streaming/examples/JavaKafkaWordCount.java  |  23 +-
 .../examples/JavaNetworkWordCount.java          |  15 +-
 .../streaming/examples/JavaQueueStream.java     |  11 +-
 .../streaming/examples/ActorWordCount.scala     |   4 +-
 .../streaming/examples/FlumeEventCount.scala    |   3 +-
 .../streaming/examples/HdfsWordCount.scala      |   2 +-
 .../streaming/examples/KafkaWordCount.scala     |   5 +-
 .../streaming/examples/MQTTWordCount.scala      |  12 +-
 .../streaming/examples/NetworkWordCount.scala   |   2 +-
 .../examples/StatefulNetworkWordCount.scala     |   2 +-
 .../streaming/examples/TwitterAlgebirdCMS.scala |   6 +-
 .../streaming/examples/TwitterAlgebirdHLL.scala |   4 +-
 .../streaming/examples/TwitterPopularTags.scala |   3 +-
 .../streaming/examples/ZeroMQWordCount.scala    |  13 +-
 .../clickstream/PageViewGenerator.scala         |   4 +-
 .../examples/clickstream/PageViewStream.scala   |   4 +-
 external/flume/pom.xml                          |  93 ++++++
 .../streaming/flume/FlumeInputDStream.scala     | 155 +++++++++
 .../spark/streaming/flume/FlumeUtils.scala      |  70 ++++
 .../streaming/flume/JavaFlumeStreamSuite.java   |  34 ++
 .../flume/src/test/resources/log4j.properties   |  29 ++
 .../streaming/flume/FlumeStreamSuite.scala      |  86 +++++
 external/kafka/pom.xml                          |  97 ++++++
 .../streaming/kafka/KafkaInputDStream.scala     | 154 +++++++++
 .../spark/streaming/kafka/KafkaUtils.scala      | 153 +++++++++
 .../streaming/kafka/JavaKafkaStreamSuite.java   |  45 +++
 .../kafka/src/test/resources/log4j.properties   |  29 ++
 .../streaming/kafka/KafkaStreamSuite.scala      |  39 +++
 external/mqtt/pom.xml                           | 108 ++++++
 .../spark/streaming/mqtt/MQTTInputDStream.scala | 110 +++++++
 .../apache/spark/streaming/mqtt/MQTTUtils.scala |  75 +++++
 .../streaming/mqtt/JavaMQTTStreamSuite.java     |  37 +++
 .../mqtt/src/test/resources/log4j.properties    |  29 ++
 .../spark/streaming/mqtt/MQTTStreamSuite.scala  |  36 ++
 external/twitter/pom.xml                        |  89 +++++
 .../streaming/twitter/TwitterInputDStream.scala | 100 ++++++
 .../spark/streaming/twitter/TwitterUtils.scala  | 126 +++++++
 .../twitter/JavaTwitterStreamSuite.java         |  46 +++
 .../twitter/src/test/resources/log4j.properties |  29 ++
 .../streaming/twitter/TwitterStreamSuite.scala  |  43 +++
 external/zeromq/pom.xml                         |  89 +++++
 .../spark/streaming/zeromq/ZeroMQReceiver.scala |  54 +++
 .../spark/streaming/zeromq/ZeroMQUtils.scala    | 126 +++++++
 .../streaming/zeromq/JavaZeroMQStreamSuite.java |  50 +++
 .../zeromq/src/test/resources/log4j.properties  |  29 ++
 .../streaming/zeromq/ZeroMQStreamSuite.scala    |  44 +++
 make-distribution.sh                            |   9 +-
 .../spark/mllib/api/python/PythonMLLibAPI.scala |  30 ++
 .../spark/mllib/optimization/Gradient.scala     |   4 +-
 .../MatrixFactorizationModel.scala              |  38 ++-
 .../optimization/GradientDescentSuite.scala     | 116 +++++++
 .../spark/mllib/recommendation/ALSSuite.scala   |  33 +-
 pom.xml                                         |  30 +-
 project/SparkBuild.scala                        |  93 ++++--
 project/build.properties                        |   1 -
 python/pyspark/mllib/_common.py                 |  25 ++
 python/pyspark/mllib/recommendation.py          |  12 +-
 .../apache/spark/repl/SparkCommandLine.scala    |  37 +++
 .../org/apache/spark/repl/SparkILoop.scala      |   7 +-
 .../apache/spark/repl/SparkRunnerSettings.scala |  32 ++
 sbin/stop-slaves.sh                             |   4 +-
 sbt/sbt                                         |  61 ++++
 streaming/pom.xml                               |  83 ++---
 .../spark/streaming/StreamingContext.scala      | 145 +-------
 .../api/java/JavaStreamingContext.scala         | 243 +-------------
 .../streaming/dstream/FlumeInputDStream.scala   | 154 ---------
 .../streaming/dstream/KafkaInputDStream.scala   | 153 ---------
 .../streaming/dstream/MQTTInputDStream.scala    | 110 -------
 .../streaming/dstream/TwitterInputDStream.scala |  99 ------
 .../streaming/receivers/ZeroMQReceiver.scala    |  53 ---
 .../scheduler/NetworkInputTracker.scala         |   4 +-
 .../apache/spark/streaming/JavaAPISuite.java    | 104 +-----
 .../streaming/LocalJavaStreamingContext.java    |  46 +++
 .../spark/streaming/InputStreamsSuite.scala     |  82 +----
 .../apache/spark/streaming/TestSuiteBase.scala  |  20 +-
 134 files changed, 3485 insertions(+), 1601 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/bc81ce04/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
----------------------------------------------------------------------
diff --cc core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index 62567a2,28ebbdc..ffa909c
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@@ -48,10 -48,10 +48,10 @@@ private[spark] object TestClient 
      val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0,
        conf = new SparkConf)
      val desc = new ApplicationDescription(
-       "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()),
+       "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()),
        "dummy-spark-home", "ignored")
      val listener = new TestListener
 -    val client = new Client(actorSystem, Array(url), desc, listener, new SparkConf)
 +    val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf)
      client.start()
      actorSystem.awaitTermination()
    }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/bc81ce04/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/bc81ce04/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/bc81ce04/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
----------------------------------------------------------------------
diff --cc core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index 028196f,331fa3a..d05bbd6
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@@ -72,34 -68,20 +72,36 @@@ class JsonProtocolSuite extends FunSuit
      assertValidJson(output)
    }
  
 -  def createAppDesc() : ApplicationDescription = {
 +  def createAppDesc(): ApplicationDescription = {
      val cmd = new Command("mainClass", List("arg1", "arg2"), Map())
-     new ApplicationDescription("name", 4, 1234, cmd, "sparkHome", "appUiUrl")
+     new ApplicationDescription("name", Some(4), 1234, cmd, "sparkHome", "appUiUrl")
    }
-   def createAppInfo(): ApplicationInfo = {
-     new ApplicationInfo(3, "id", createAppDesc(), new Date(123456789), null, "appUriStr")
++
+   def createAppInfo() : ApplicationInfo = {
+     new ApplicationInfo(
+       3, "id", createAppDesc(), new Date(123456789), null, "appUriStr", Int.MaxValue)
    }
 -  def createWorkerInfo() : WorkerInfo = {
 +
 +  def createDriverCommand() = new Command(
 +    "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"),
 +    Map(("K1", "V1"), ("K2", "V2"))
 +  )
 +
 +  def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3,
 +    false, createDriverCommand())
 +
 +  def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", createDriverDesc(), new Date())
 +
 +  def createWorkerInfo(): WorkerInfo = {
      new WorkerInfo("id", "host", 8080, 4, 1234, null, 80, "publicAddress")
    }
 -  def createExecutorRunner() : ExecutorRunner = {
 +  def createExecutorRunner(): ExecutorRunner = {
      new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host",
 -      new File("sparkHome"), new File("workDir"), ExecutorState.RUNNING)
 +      new File("sparkHome"), new File("workDir"), "akka://worker", ExecutorState.RUNNING)
 +  }
 +  def createDriverRunner(): DriverRunner = {
 +    new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), createDriverDesc(),
 +      null, "akka://worker")
    }
  
    def assertValidJson(json: JValue) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/bc81ce04/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/bc81ce04/docs/spark-standalone.md
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/bc81ce04/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 7b734c5,c8e3737..9f08e8a
--- a/pom.xml
+++ b/pom.xml
@@@ -259,29 -253,6 +253,17 @@@
          </exclusions>
        </dependency>
        <dependency>
 +        <groupId>${akka.group}</groupId>
-         <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
-         <version>${akka.version}</version>
-         <exclusions>
-           <exclusion>
-             <groupId>org.jboss.netty</groupId>
-             <artifactId>netty</artifactId>
-           </exclusion>
-         </exclusions>
-       </dependency>
-       <dependency>
-         <groupId>${akka.group}</groupId>
 +        <artifactId>akka-testkit_${scala.binary.version}</artifactId>
 +        <version>${akka.version}</version>
-         <scope>test</scope>
 +        <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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/bc81ce04/project/SparkBuild.scala
----------------------------------------------------------------------


[11/37] git commit: Updated approach to driver restarting

Posted by pw...@apache.org.
Updated approach to driver restarting


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

Branch: refs/heads/master
Commit: 5938cfc153ac79148f288a1f228458a5df0c74dc
Parents: bbc3628
Author: Patrick Wendell <pw...@gmail.com>
Authored: Thu Dec 26 12:02:19 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Thu Dec 26 12:02:19 2013 -0800

----------------------------------------------------------------------
 .../spark/deploy/worker/DriverRunner.scala      | 45 ++++++++++----------
 .../spark/deploy/worker/DriverWrapper.scala     |  8 ++++
 2 files changed, 30 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5938cfc1/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index b9b6341..d13e651 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -41,7 +41,7 @@ private[spark] class DriverRunner(
     val worker: ActorRef)
   extends Logging {
 
-  var process: Option[Process] = None
+  @volatile var process: Option[Process] = None
   @volatile var killed = false
 
   /** Starts a thread to run and manage the driver. */
@@ -73,8 +73,10 @@ private[spark] class DriverRunner(
 
   /** Terminate this driver (or prevent it from ever starting if not yet started) */
   def kill() {
-    killed = true
-    process.foreach(p => p.destroy())
+    synchronized {
+      process.foreach(p => p.destroy())
+      killed = true
+    }
   }
 
   /** Spawn a thread that will redirect a given stream to a file */
@@ -134,43 +136,40 @@ private[spark] class DriverRunner(
   }
 
   /** Continue launching the supplied command until it exits zero or is killed. */
-  def runCommandWithRetry(command: Seq[String], envVars: Seq[(String, String)], baseDir: File) = {
+  def runCommandWithRetry(command: Seq[String], envVars: Seq[(String, String)], baseDir: File)
+      : Unit = {
     // Time to wait between submission retries.
     var waitSeconds = 1
     var cleanExit = false
 
     while (!cleanExit && !killed) {
-      Thread.sleep(waitSeconds * 1000)
-
       logInfo("Launch Command: " + command.mkString("\"", "\" \"", "\""))
       val builder = new ProcessBuilder(command: _*).directory(baseDir)
       envVars.map{ case(k,v) => builder.environment().put(k, v) }
 
-      process = Some(builder.start())
+      synchronized {
+        if (killed) { return }
 
-      // Redirect stdout and stderr to files
-      val stdout = new File(baseDir, "stdout")
-      redirectStream(process.get.getInputStream, stdout)
+        process = Some(builder.start())
 
-      val stderr = new File(baseDir, "stderr")
-      val header = "Launch Command: %s\n%s\n\n".format(
-        command.mkString("\"", "\" \"", "\""), "=" * 40)
-      Files.write(header, stderr, Charsets.UTF_8)
-      redirectStream(process.get.getErrorStream, stderr)
+        // Redirect stdout and stderr to files
+        val stdout = new File(baseDir, "stdout")
+        redirectStream(process.get.getInputStream, stdout)
+
+        val stderr = new File(baseDir, "stderr")
+        val header = "Launch Command: %s\n%s\n\n".format(
+          command.mkString("\"", "\" \"", "\""), "=" * 40)
+        Files.write(header, stderr, Charsets.UTF_8)
+        redirectStream(process.get.getErrorStream, stderr)
+      }
 
-      val exitCode =
-        /* There is a race here I've elected to ignore for now because it's very unlikely and not
-         * simple to fix. This could see `killed=false` then the main thread gets a kill request
-         * and sets `killed=true` and destroys the not-yet-started process, then this thread
-         * launches the process. For now, in that case the user can just re-submit the kill
-         * request. */
-        if (killed) -1
-        else process.get.waitFor()
+      val exitCode = process.get.waitFor()
 
       cleanExit = exitCode == 0
       if (!cleanExit && !killed) {
         waitSeconds = waitSeconds * 2 // exponential back-off
         logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.")
+        (0 until waitSeconds).takeWhile(f => {Thread.sleep(1000); !killed})
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5938cfc1/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
new file mode 100644
index 0000000..92fb084
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -0,0 +1,8 @@
+package org.apache.spark.deploy.worker
+
+object DriverWrapper {
+  def main(args: Array[String]) {
+    val c = Console.readChar()
+    println(s"Char: $c")
+  }
+}


[28/37] git commit: Adding unit tests and some refactoring to promote testability.

Posted by pw...@apache.org.
Adding unit tests and some refactoring to promote testability.


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

Branch: refs/heads/master
Commit: e21a707a13b437327cef25d44ef08ddb2e3931af
Parents: b72cceb
Author: Patrick Wendell <pw...@gmail.com>
Authored: Tue Jan 7 00:21:43 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Tue Jan 7 15:39:47 2014 -0800

----------------------------------------------------------------------
 core/pom.xml                                    |   5 +
 .../spark/deploy/worker/DriverRunner.scala      |  88 +++++++++----
 .../spark/deploy/worker/ExecutorRunner.scala    |  10 +-
 .../spark/deploy/worker/WorkerWatcher.scala     |  14 +-
 .../apache/spark/deploy/JsonProtocolSuite.scala |   2 +-
 .../spark/deploy/worker/DriverRunnerTest.scala  | 131 +++++++++++++++++++
 .../deploy/worker/ExecutorRunnerTest.scala      |   4 +-
 .../deploy/worker/WorkerWatcherSuite.scala      |  32 +++++
 pom.xml                                         |  12 ++
 project/SparkBuild.scala                        |   1 +
 10 files changed, 264 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index aac0a9d..1c52b33 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -99,6 +99,11 @@
             <artifactId>akka-slf4j_${scala.binary.version}</artifactId>
         </dependency>
         <dependency>
+            <groupId>${akka.group}</groupId>
+            <artifactId>akka-testkit_${scala.binary.version}</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>org.scala-lang</groupId>
             <artifactId>scala-library</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index f726089..d13d7ef 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -19,6 +19,7 @@ package org.apache.spark.deploy.worker
 
 import java.io._
 
+import scala.collection.JavaConversions._
 import scala.collection.mutable.Map
 
 import akka.actor.ActorRef
@@ -47,6 +48,16 @@ private[spark] class DriverRunner(
   @volatile var process: Option[Process] = None
   @volatile var killed = false
 
+  // Decoupled for testing
+  private[deploy] def setClock(_clock: Clock) = clock = _clock
+  private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper
+  private var clock = new Clock {
+    def currentTimeMillis(): Long = System.currentTimeMillis()
+  }
+  private var sleeper = new Sleeper {
+    def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed})
+  }
+
   /** Starts a thread to run and manage the driver. */
   def start() = {
     new Thread("DriverRunner for " + driverId) {
@@ -63,10 +74,9 @@ private[spark] class DriverRunner(
           env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename"
           val newCommand = Command(driverDesc.command.mainClass,
             driverDesc.command.arguments.map(substituteVariables), env)
-
           val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem,
             sparkHome.getAbsolutePath)
-          runCommand(command, env, driverDir, driverDesc.supervise)
+          launchDriver(command, env, driverDir, driverDesc.supervise)
         }
         catch {
           case e: Exception => exn = Some(e)
@@ -116,7 +126,7 @@ private[spark] class DriverRunner(
 
     val jarPath = new Path(driverDesc.jarUrl)
 
-    val emptyConf = new Configuration() // TODO: In docs explain it needs to be full HDFS path
+    val emptyConf = new Configuration()
     val jarFileSystem = jarPath.getFileSystem(emptyConf)
 
     val destPath = new File(driverDir.getAbsolutePath, jarPath.getName)
@@ -136,51 +146,77 @@ private[spark] class DriverRunner(
     localJarFilename
   }
 
-  /** Launch the supplied command. */
-  private def runCommand(command: Seq[String], envVars: Map[String, String], baseDir: File,
-      supervise: Boolean) {
+  private def launchDriver(command: Seq[String], envVars: Map[String, String], baseDir: File,
+                           supervise: Boolean) {
+    val builder = new ProcessBuilder(command: _*).directory(baseDir)
+    envVars.map{ case(k,v) => builder.environment().put(k, v) }
+
+    def initialize(process: Process) = {
+      // Redirect stdout and stderr to files
+      val stdout = new File(baseDir, "stdout")
+      CommandUtils.redirectStream(process.getInputStream, stdout)
+
+      val stderr = new File(baseDir, "stderr")
+      val header = "Launch Command: %s\n%s\n\n".format(
+        command.mkString("\"", "\" \"", "\""), "=" * 40)
+      Files.append(header, stderr, Charsets.UTF_8)
+      CommandUtils.redirectStream(process.getErrorStream, stderr)
+    }
+    runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise)
+  }
 
+  private[deploy] def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit,
+    supervise: Boolean) {
     // Time to wait between submission retries.
     var waitSeconds = 1
     // A run of this many seconds resets the exponential back-off.
-    val successfulRunDuration = 1
+    val successfulRunDuration = 5
 
     var keepTrying = !killed
 
     while (keepTrying) {
-      logInfo("Launch Command: " + command.mkString("\"", "\" \"", "\""))
-      val builder = new ProcessBuilder(command: _*).directory(baseDir)
-      envVars.map{ case(k,v) => builder.environment().put(k, v) }
+      logInfo("Launch Command: " + command.command.mkString("\"", "\" \"", "\""))
 
       synchronized {
         if (killed) { return }
-
-        process = Some(builder.start())
-
-        // Redirect stdout and stderr to files
-        val stdout = new File(baseDir, "stdout")
-        CommandUtils.redirectStream(process.get.getInputStream, stdout)
-
-        val stderr = new File(baseDir, "stderr")
-        val header = "Launch Command: %s\n%s\n\n".format(
-          command.mkString("\"", "\" \"", "\""), "=" * 40)
-        Files.append(header, stderr, Charsets.UTF_8)
-        CommandUtils.redirectStream(process.get.getErrorStream, stderr)
+        process = Some(command.start())
+        initialize(process.get)
       }
 
-      val processStart = System.currentTimeMillis()
+      val processStart = clock.currentTimeMillis()
       val exitCode = process.get.waitFor()
-      if (System.currentTimeMillis() - processStart > successfulRunDuration * 1000) {
+      if (clock.currentTimeMillis() - processStart > successfulRunDuration * 1000) {
         waitSeconds = 1
       }
 
       if (supervise && exitCode != 0 && !killed) {
-        waitSeconds = waitSeconds * 2 // exponential back-off
         logInfo(s"Command exited with status $exitCode, re-launching after $waitSeconds s.")
-        (0 until waitSeconds).takeWhile(f => {Thread.sleep(1000); !killed})
+        sleeper.sleep(waitSeconds)
+        waitSeconds = waitSeconds * 2 // exponential back-off
       }
 
       keepTrying = supervise && exitCode != 0 && !killed
     }
   }
 }
+
+private[deploy] trait Clock {
+  def currentTimeMillis(): Long
+}
+
+private[deploy] trait Sleeper {
+  def sleep(seconds: Int)
+}
+
+// Needed because ProcessBuilder is a final class and cannot be mocked
+private[deploy] trait ProcessBuilderLike {
+  def start(): Process
+  def command: Seq[String]
+}
+
+private[deploy] object ProcessBuilderLike {
+  def apply(processBuilder: ProcessBuilder) = new ProcessBuilderLike {
+    def start() = processBuilder.start()
+    def command = processBuilder.command()
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index fdc9a34..a9cb998 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -98,6 +98,12 @@ private[spark] class ExecutorRunner(
     case other => other
   }
 
+  def getCommandSeq = {
+    val command = Command(appDesc.command.mainClass,
+      appDesc.command.arguments.map(substituteVariables), appDesc.command.environment)
+    CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath)
+  }
+
   /**
    * Download and run the executor described in our ApplicationDescription
    */
@@ -110,9 +116,7 @@ private[spark] class ExecutorRunner(
       }
 
       // Launch the process
-      val fullCommand = new Command(appDesc.command.mainClass,
-        appDesc.command.arguments.map(substituteVariables), appDesc.command.environment)
-      val command = CommandUtils.buildCommandSeq(fullCommand, memory, sparkHome.getAbsolutePath)
+      val command = getCommandSeq
       logInfo("Launch command: " + command.mkString("\"", "\" \"", "\""))
       val builder = new ProcessBuilder(command: _*).directory(executorDir)
       val env = builder.environment()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
index f4184bc..0e0d0cd 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
@@ -10,7 +10,8 @@ import org.apache.spark.deploy.DeployMessages.SendHeartbeat
  * Actor which connects to a worker process and terminates the JVM if the connection is severed.
  * Provides fate sharing between a worker and its associated child processes.
  */
-private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging {
+private[spark] class WorkerWatcher(workerUrl: String) extends Actor
+    with Logging {
   override def preStart() {
     context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
 
@@ -19,10 +20,17 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging
     worker ! SendHeartbeat // need to send a message here to initiate connection
   }
 
+  // Used to avoid shutting down JVM during tests
+  private[deploy] var isShutDown = false
+  private[deploy] def setTesting(testing: Boolean) = isTesting = testing
+  private var isTesting = false
+
   // Lets us filter events only from the worker's actor system
   private val expectedHostPort = AddressFromURIString(workerUrl).hostPort
   private def isWorker(address: Address) = address.hostPort == expectedHostPort
 
+  def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1)
+
   override def receive = {
     case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) =>
       logInfo(s"Successfully connected to $workerUrl")
@@ -32,12 +40,12 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor with Logging
       // These logs may not be seen if the worker (and associated pipe) has died
       logError(s"Could not initialize connection to worker $workerUrl. Exiting.")
       logError(s"Error was: $cause")
-      System.exit(-1)
+      exitNonZero()
 
     case DisassociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) =>
       // This log message will never be seen
       logError(s"Lost connection to worker actor $workerUrl. Exiting.")
-      System.exit(-1)
+      exitNonZero()
 
     case e: AssociationEvent =>
       // pass through association events relating to other remote actor systems

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index 372c9f4..028196f 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -86,7 +86,7 @@ class JsonProtocolSuite extends FunSuite {
   )
 
   def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3,
-    createDriverCommand())
+    false, createDriverCommand())
 
   def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", createDriverDesc(), new Date())
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
new file mode 100644
index 0000000..45dbcaf
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
@@ -0,0 +1,131 @@
+package org.apache.spark.deploy.worker
+
+import java.io.File
+
+import scala.collection.JavaConversions._
+
+import org.mockito.Mockito._
+import org.mockito.Matchers._
+import org.scalatest.FunSuite
+
+import org.apache.spark.deploy.{Command, DriverDescription}
+import org.mockito.stubbing.Answer
+import org.mockito.invocation.InvocationOnMock
+
+class DriverRunnerTest extends FunSuite {
+  private def createDriverRunner() = {
+    val command = new Command("mainClass", Seq(), Map())
+    val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command)
+    new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription,
+      null, "akka://1.2.3.4/worker/")
+  }
+
+  private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = {
+    val processBuilder = mock(classOf[ProcessBuilderLike])
+    when(processBuilder.command).thenReturn(Seq("mocked", "command"))
+    val process = mock(classOf[Process])
+    when(processBuilder.start()).thenReturn(process)
+    (processBuilder, process)
+  }
+
+  test("Process succeeds instantly") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+    // One failure then a successful run
+    when(process.waitFor()).thenReturn(0)
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = true)
+
+    verify(process, times(1)).waitFor()
+    verify(sleeper, times(0)).sleep(anyInt())
+  }
+
+  test("Process failing several times and then succeeding") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+    // fail, fail, fail, success
+    when(process.waitFor()).thenReturn(-1).thenReturn(-1).thenReturn(-1).thenReturn(0)
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = true)
+
+    verify(process, times(4)).waitFor()
+    verify(sleeper, times(3)).sleep(anyInt())
+    verify(sleeper, times(1)).sleep(1)
+    verify(sleeper, times(1)).sleep(2)
+    verify(sleeper, times(1)).sleep(4)
+  }
+
+  test("Process doesn't restart if not supervised") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+    when(process.waitFor()).thenReturn(-1)
+
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = false)
+
+    verify(process, times(1)).waitFor()
+    verify(sleeper, times(0)).sleep(anyInt())
+  }
+
+  test("Process doesn't restart if killed") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+    when(process.waitFor()).thenAnswer(new Answer[Int] {
+      def answer(invocation: InvocationOnMock): Int = {
+        runner.kill()
+        -1
+      }
+    })
+
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = true)
+
+    verify(process, times(1)).waitFor()
+    verify(sleeper, times(0)).sleep(anyInt())
+  }
+
+  test("Reset of backoff counter") {
+    val runner = createDriverRunner()
+
+    val sleeper = mock(classOf[Sleeper])
+    runner.setSleeper(sleeper)
+
+    val clock = mock(classOf[Clock])
+    runner.setClock(clock)
+
+    val (processBuilder, process) = createProcessBuilderAndProcess()
+
+    when(process.waitFor())
+      .thenReturn(-1) // fail 1
+      .thenReturn(-1) // fail 2
+      .thenReturn(-1) // fail 3
+      .thenReturn(-1) // fail 4
+      .thenReturn(0) // success
+    when(clock.currentTimeMillis())
+      .thenReturn(0).thenReturn(1000) // fail 1 (short)
+      .thenReturn(1000).thenReturn(2000) // fail 2 (short)
+      .thenReturn(2000).thenReturn(10000) // fail 3 (long)
+      .thenReturn(10000).thenReturn(11000) // fail 4 (short)
+      .thenReturn(11000).thenReturn(21000) // success (long)
+
+    runner.runCommandWithRetry(processBuilder, p => (), supervise = true)
+
+    verify(sleeper, times(4)).sleep(anyInt())
+    // Expected sequence of sleeps is 1,2,1,2
+    verify(sleeper, times(2)).sleep(1)
+    verify(sleeper, times(2)).sleep(2)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
index 7e5aaa3..bdb2c86 100644
--- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
@@ -31,8 +31,8 @@ class ExecutorRunnerTest extends FunSuite {
       sparkHome, "appUiUrl")
     val appId = "12345-worker321-9876"
     val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome),
-      f("ooga"), ExecutorState.RUNNING)
+      f("ooga"), "blah", ExecutorState.RUNNING)
 
-    assert(er.buildCommandSeq().last === appId)
+    assert(er.getCommandSeq.last === appId)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala
new file mode 100644
index 0000000..94d88d3
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala
@@ -0,0 +1,32 @@
+package org.apache.spark.deploy.worker
+
+
+import akka.testkit.TestActorRef
+import org.scalatest.FunSuite
+import akka.remote.DisassociatedEvent
+import akka.actor.{ActorSystem, AddressFromURIString, Props}
+
+class WorkerWatcherSuite extends FunSuite {
+  test("WorkerWatcher shuts down on valid disassociation") {
+    val actorSystem = ActorSystem("test")
+    val targetWorkerUrl = "akka://1.2.3.4/user/Worker"
+    val targetWorkerAddress = AddressFromURIString(targetWorkerUrl)
+    val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem)
+    val workerWatcher = actorRef.underlyingActor
+    workerWatcher.setTesting(testing = true)
+    actorRef.underlyingActor.receive(new DisassociatedEvent(null, targetWorkerAddress, false))
+    assert(actorRef.underlyingActor.isShutDown)
+  }
+
+  test("WorkerWatcher stays alive on invalid disassociation") {
+    val actorSystem = ActorSystem("test")
+    val targetWorkerUrl = "akka://1.2.3.4/user/Worker"
+    val otherAkkaURL = "akka://4.3.2.1/user/OtherActor"
+    val otherAkkaAddress = AddressFromURIString(otherAkkaURL)
+    val actorRef = TestActorRef[WorkerWatcher](Props(classOf[WorkerWatcher], targetWorkerUrl))(actorSystem)
+    val workerWatcher = actorRef.underlyingActor
+    workerWatcher.setTesting(testing = true)
+    actorRef.underlyingActor.receive(new DisassociatedEvent(null, otherAkkaAddress, false))
+    assert(!actorRef.underlyingActor.isShutDown)
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 78d2f16..7b734c5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -270,6 +270,18 @@
         </exclusions>
       </dependency>
       <dependency>
+        <groupId>${akka.group}</groupId>
+        <artifactId>akka-testkit_${scala.binary.version}</artifactId>
+        <version>${akka.version}</version>
+        <scope>test</scope>
+        <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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e21a707a/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 051e510..bd5f3f7 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -233,6 +233,7 @@ object SparkBuild extends Build {
         "org.ow2.asm"              % "asm"              % "4.0",
         "org.spark-project.akka"  %% "akka-remote"      % "2.2.3-shaded-protobuf"  excludeAll(excludeNetty),
         "org.spark-project.akka"  %% "akka-slf4j"       % "2.2.3-shaded-protobuf"  excludeAll(excludeNetty),
+        "org.spark-project.akka"  %% "akka-testkit"     % "2.2.3-shaded-protobuf" % "test",
         "net.liftweb"             %% "lift-json"        % "2.5.1"  excludeAll(excludeNetty),
         "it.unimi.dsi"             % "fastutil"         % "6.4.4",
         "colt"                     % "colt"             % "1.2.0",


[35/37] git commit: Adding polling to driver submission client.

Posted by pw...@apache.org.
Adding polling to driver submission client.


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

Branch: refs/heads/master
Commit: 0f9d2ace6baefeacb1abf9d51a457644b67f2f8d
Parents: 62b08fa
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Jan 8 16:53:04 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Jan 8 16:56:26 2014 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/deploy/Client.scala  | 141 ++++++++++++-------
 .../org/apache/spark/deploy/DeployMessage.scala |  11 +-
 .../spark/deploy/master/DriverState.scala       |   5 +-
 .../org/apache/spark/deploy/master/Master.scala |  29 ++--
 .../spark/deploy/worker/DriverRunner.scala      |  12 +-
 .../org/apache/spark/deploy/worker/Worker.scala |   2 +-
 6 files changed, 132 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0f9d2ace/core/src/main/scala/org/apache/spark/deploy/Client.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala
index 43b9b1c..e133893 100644
--- a/core/src/main/scala/org/apache/spark/deploy/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala
@@ -22,60 +22,30 @@ import scala.collection.mutable.Map
 import scala.concurrent._
 
 import akka.actor._
+import akka.pattern.ask
 import org.apache.log4j.{Level, Logger}
 
 import org.apache.spark.{Logging, SparkConf}
 import org.apache.spark.deploy.DeployMessages._
-import org.apache.spark.deploy.master.Master
+import org.apache.spark.deploy.master.{DriverState, Master}
 import org.apache.spark.util.{AkkaUtils, Utils}
+import akka.actor.Actor.emptyBehavior
+import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent}
 
 /**
- * Actor that sends a single message to the standalone master and returns the response in the
- * given promise.
+ * Proxy that relays messages to the driver.
  */
-class DriverActor(master: String, response: Promise[(Boolean, String)]) extends Actor with Logging {
-  override def receive = {
-    case SubmitDriverResponse(success, message) => {
-      response.success((success, message))
-    }
-
-    case KillDriverResponse(success, message) => {
-      response.success((success, message))
-    }
+class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends Actor with Logging {
+  var masterActor: ActorSelection = _
+  val timeout = AkkaUtils.askTimeout(conf)
 
-    // Relay all other messages to the master.
-    case message => {
-      logInfo(s"Sending message to master $master...")
-      val masterActor = context.actorSelection(Master.toAkkaUrl(master))
-      masterActor ! message
-    }
-  }
-}
+  override def preStart() = {
+    masterActor = context.actorSelection(Master.toAkkaUrl(driverArgs.master))
 
-/**
- * Executable utility for starting and terminating drivers inside of a standalone cluster.
- */
-object Client {
-
-  def main(args: Array[String]) {
-    val driverArgs = new ClientArguments(args)
-    val conf = new SparkConf()
-
-    if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) {
-      conf.set("spark.akka.logLifecycleEvents", "true")
-    }
-    conf.set("spark.akka.askTimeout", "5")
-    Logger.getRootLogger.setLevel(driverArgs.logLevel)
-
-    // TODO: See if we can initialize akka so return messages are sent back using the same TCP
-    //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
-    val (actorSystem, _) = AkkaUtils.createActorSystem(
-      "driverClient", Utils.localHostName(), 0, false, conf)
-    val master = driverArgs.master
-    val response = promise[(Boolean, String)]
-    val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response)))
+    context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
 
     println(s"Sending ${driverArgs.cmd} command to ${driverArgs.master}")
+
     driverArgs.cmd match {
       case "launch" =>
         // TODO: We could add an env variable here and intercept it in `sc.addJar` that would
@@ -94,21 +64,88 @@ object Client {
           driverArgs.cores,
           driverArgs.supervise,
           command)
-        driver ! RequestSubmitDriver(driverDescription)
+
+        masterActor ! RequestSubmitDriver(driverDescription)
 
       case "kill" =>
         val driverId = driverArgs.driverId
-        driver ! RequestKillDriver(driverId)
+        val killFuture = masterActor ! RequestKillDriver(driverId)
+    }
+  }
+
+  /* Find out driver status then exit the JVM */
+  def pollAndReportStatus(driverId: String) {
+    println(s"... waiting before polling master for driver state")
+    Thread.sleep(5000)
+    println("... polling master for driver state")
+    val statusFuture = (masterActor ? RequestDriverStatus(driverId))(timeout)
+      .mapTo[DriverStatusResponse]
+    val statusResponse = Await.result(statusFuture, timeout)
+
+    statusResponse.found match {
+      case false =>
+        println(s"ERROR: Cluster master did not recognize $driverId")
+        System.exit(-1)
+      case true =>
+        println(s"State of $driverId is ${statusResponse.state.get}")
+        // Worker node, if present
+        (statusResponse.workerId, statusResponse.workerHostPort, statusResponse.state) match {
+          case (Some(id), Some(hostPort), Some(DriverState.RUNNING)) =>
+            println(s"Driver running on $hostPort ($id)")
+          case _ =>
+        }
+        // Exception, if present
+        statusResponse.exception.map { e =>
+          println(s"Exception from cluster was: $e")
+          System.exit(-1)
+        }
+        System.exit(0)
     }
+  }
+
+  override def receive = {
+
+    case SubmitDriverResponse(success, driverId, message) =>
+      println(message)
+      if (success) pollAndReportStatus(driverId.get) else System.exit(-1)
+
+    case KillDriverResponse(driverId, success, message) =>
+      println(message)
+      if (success) pollAndReportStatus(driverId) else System.exit(-1)
+
+    case DisassociatedEvent(_, remoteAddress, _) =>
+      println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.")
+      System.exit(-1)
+
+    case AssociationErrorEvent(cause, _, remoteAddress, _) =>
+      println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.")
+      println(s"Cause was: $cause")
+      System.exit(-1)
+  }
+}
+
+/**
+ * Executable utility for starting and terminating drivers inside of a standalone cluster.
+ */
+object Client {
+  def main(args: Array[String]) {
+    val conf = new SparkConf()
+    val driverArgs = new ClientArguments(args)
+
+    if (!driverArgs.logLevel.isGreaterOrEqual(Level.WARN)) {
+      conf.set("spark.akka.logLifecycleEvents", "true")
+    }
+    conf.set("spark.akka.askTimeout", "10")
+    conf.set("akka.loglevel", driverArgs.logLevel.toString.replace("WARN", "WARNING"))
+    Logger.getRootLogger.setLevel(driverArgs.logLevel)
+
+    // TODO: See if we can initialize akka so return messages are sent back using the same TCP
+    //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
+    val (actorSystem, _) = AkkaUtils.createActorSystem(
+      "driverClient", Utils.localHostName(), 0, false, conf)
+
+    actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf))
 
-    val (success, message) =
-      try {
-        Await.result(response.future, AkkaUtils.askTimeout(conf))
-      } catch {
-        case e: TimeoutException => (false, s"Error: Timed out sending message to $master")
-      }
-    println(message)
-    actorSystem.shutdown()
     actorSystem.awaitTermination()
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0f9d2ace/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 34460d3..5e824e1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -112,11 +112,18 @@ private[deploy] object DeployMessages {
 
   case class RequestSubmitDriver(driverDescription: DriverDescription) extends DeployMessage
 
-  case class SubmitDriverResponse(success: Boolean, message: String) extends DeployMessage
+  case class SubmitDriverResponse(success: Boolean, driverId: Option[String], message: String)
+    extends DeployMessage
 
   case class RequestKillDriver(driverId: String) extends DeployMessage
 
-  case class KillDriverResponse(success: Boolean, message: String) extends DeployMessage
+  case class KillDriverResponse(driverId: String, success: Boolean, message: String)
+    extends DeployMessage
+
+  case class RequestDriverStatus(driverId: String) extends DeployMessage
+
+  case class DriverStatusResponse(found: Boolean, state: Option[DriverState],
+    workerId: Option[String], workerHostPort: Option[String], exception: Option[Exception])
 
   // Internal message in AppClient
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0f9d2ace/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
index 93b2607..26a68ba 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala
@@ -27,6 +27,7 @@ private[spark] object DriverState extends Enumeration {
   // RELAUNCHING: Exited non-zero or due to worker failure, but has not yet started running again
   // UNKNOWN: The state of the driver is temporarily not known due to master failure recovery
   // KILLED: A user manually killed this driver
-  // FAILED: Unable to run due to an unrecoverable error (e.g. missing jar file)
-  val SUBMITTED, RUNNING, FINISHED, RELAUNCHING, UNKNOWN, KILLED, FAILED = Value
+  // FAILED: The driver exited non-zero and was not supervised
+  // ERROR: Unable to run or restart due to an unrecoverable error (e.g. missing jar file)
+  val SUBMITTED, RUNNING, FINISHED, RELAUNCHING, UNKNOWN, KILLED, FAILED, ERROR = Value
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0f9d2ace/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index f62601f..cd3f3eb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -186,7 +186,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     case RequestSubmitDriver(description) => {
       if (state != RecoveryState.ALIVE) {
         val msg = s"Can only accept driver submissions in ALIVE state. Current state: $state."
-        sender ! SubmitDriverResponse(false, msg)
+        sender ! SubmitDriverResponse(false, None, msg)
       } else {
         logInfo("Driver submitted " + description.command.mainClass)
         val driver = createDriver(description)
@@ -198,14 +198,15 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         // TODO: It might be good to instead have the submission client poll the master to determine
         //       the current status of the driver. For now it's simply "fire and forget".
 
-        sender ! SubmitDriverResponse(true, s"Driver successfully submitted as ${driver.id}")
+        sender ! SubmitDriverResponse(true, Some(driver.id),
+          s"Driver successfully submitted as ${driver.id}")
       }
     }
 
     case RequestKillDriver(driverId) => {
       if (state != RecoveryState.ALIVE) {
         val msg = s"Can only kill drivers in ALIVE state. Current state: $state."
-        sender ! KillDriverResponse(false, msg)
+        sender ! KillDriverResponse(driverId, success = false, msg)
       } else {
         logInfo("Asked to kill driver " + driverId)
         val driver = drivers.find(_.id == driverId)
@@ -226,15 +227,25 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
             // TODO: It would be nice for this to be a synchronous response
             val msg = s"Kill request for $driverId submitted"
             logInfo(msg)
-            sender ! KillDriverResponse(true, msg)
+            sender ! KillDriverResponse(driverId, success = true, msg)
           case None =>
-            val msg = s"Could not find running driver $driverId"
+            val msg = s"Driver $driverId has already finished or does not exist"
             logWarning(msg)
-            sender ! KillDriverResponse(false, msg)
+            sender ! KillDriverResponse(driverId, success = false, msg)
         }
       }
     }
 
+    case RequestDriverStatus(driverId) => {
+      (drivers ++ completedDrivers).find(_.id == driverId) match {
+        case Some(driver) =>
+          sender ! DriverStatusResponse(found = true, Some(driver.state),
+            driver.worker.map(_.id), driver.worker.map(_.hostPort), driver.exception)
+        case None =>
+          sender ! DriverStatusResponse(found = false, None, None, None, None)
+      }
+    }
+
     case RegisterApplication(description) => {
       if (state == RecoveryState.STANDBY) {
         // ignore, don't send response
@@ -279,7 +290,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
 
     case DriverStateChanged(driverId, state, exception) => {
       state match {
-        case DriverState.FAILED | DriverState.FINISHED | DriverState.KILLED =>
+        case DriverState.ERROR | DriverState.FINISHED | DriverState.KILLED | DriverState.FAILED =>
           removeDriver(driverId, state, exception)
         case _ =>
           throw new Exception(s"Received unexpected state update for driver $driverId: $state")
@@ -410,7 +421,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         logWarning(s"Re-launching ${d.id}")
         relaunchDriver(d)
       } else {
-        removeDriver(d.id, DriverState.FAILED, None)
+        removeDriver(d.id, DriverState.ERROR, None)
         logWarning(s"Did not re-launch ${d.id} because it was not supervised")
       }
     }
@@ -539,7 +550,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         relaunchDriver(driver)
       } else {
         logInfo(s"Not re-launching ${driver.id} because it was not supervised")
-        removeDriver(driver.id, DriverState.FAILED, None)
+        removeDriver(driver.id, DriverState.ERROR, None)
       }
     }
     persistenceEngine.removeWorker(worker)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0f9d2ace/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index ad70345..b4df1a0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -52,6 +52,7 @@ private[spark] class DriverRunner(
   // Populated once finished
   var finalState: Option[DriverState] = None
   var finalException: Option[Exception] = None
+  var finalExitCode: Option[Int] = None
 
   // Decoupled for testing
   private[deploy] def setClock(_clock: Clock) = clock = _clock
@@ -87,8 +88,14 @@ private[spark] class DriverRunner(
 
         val state =
           if (killed) { DriverState.KILLED }
-          else if (finalException.isDefined) { DriverState.FAILED }
-          else { DriverState.FINISHED }
+          else if (finalException.isDefined) { DriverState.ERROR }
+          else {
+            finalExitCode match {
+              case Some(0) => DriverState.FINISHED
+              case _ => DriverState.FAILED
+            }
+          }
+
         finalState = Some(state)
 
         worker ! DriverStateChanged(driverId, state, finalException)
@@ -200,6 +207,7 @@ private[spark] class DriverRunner(
       }
 
       keepTrying = supervise && exitCode != 0 && !killed
+      finalExitCode = Some(exitCode)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0f9d2ace/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 2a2b7a3..273bacd 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -272,7 +272,7 @@ private[spark] class Worker(
 
     case DriverStateChanged(driverId, state, exception) => {
       state match {
-        case DriverState.FAILED =>
+        case DriverState.ERROR =>
           logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}")
         case DriverState.FINISHED =>
           logInfo(s"Driver $driverId exited successfully")


[34/37] git commit: Adding mockito to maven build

Posted by pw...@apache.org.
Adding mockito to maven build


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

Branch: refs/heads/master
Commit: 62b08faac5278d289bdaefb42ff6f65b62ce48aa
Parents: bc81ce0
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Jan 8 00:45:41 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Jan 8 00:45:41 2014 -0800

----------------------------------------------------------------------
 core/pom.xml | 5 +++++
 pom.xml      | 6 ++++++
 2 files changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/62b08faa/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 1c52b33..9e5a450 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -171,6 +171,11 @@
             <scope>test</scope>
         </dependency>
         <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>org.scalacheck</groupId>
             <artifactId>scalacheck_${scala.binary.version}</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/62b08faa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9f08e8a..c2b1a77 100644
--- a/pom.xml
+++ b/pom.xml
@@ -352,6 +352,12 @@
         <scope>test</scope>
       </dependency>
       <dependency>
+          <groupId>org.mockito</groupId>
+          <artifactId>mockito-all</artifactId>
+          <scope>test</scope>
+          <version>1.8.5</version>
+      </dependency>
+      <dependency>
         <groupId>commons-io</groupId>
         <artifactId>commons-io</artifactId>
         <version>2.4</version>


[13/37] git commit: Merge pull request #1 from aarondav/driver

Posted by pw...@apache.org.
Merge pull request #1 from aarondav/driver

Refactor DriverClient to be more Actor-based

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

Branch: refs/heads/master
Commit: da20270b839cc10d4459848f5b485ca566cd2dfb
Parents: a97ad55 61372b1
Author: Patrick Wendell <pw...@gmail.com>
Authored: Thu Dec 26 12:11:52 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Thu Dec 26 12:11:52 2013 -0800

----------------------------------------------------------------------
 .../spark/deploy/client/DriverClient.scala      | 93 +++++++-------------
 1 file changed, 31 insertions(+), 62 deletions(-)
----------------------------------------------------------------------



[17/37] git commit: Some notes and TODO about dependencies

Posted by pw...@apache.org.
Some notes and TODO about dependencies


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

Branch: refs/heads/master
Commit: c8c8b42a6fde0d59217b264bb2439751696c467f
Parents: 55c8bb7
Author: Patrick Wendell <pw...@gmail.com>
Authored: Fri Dec 27 15:13:11 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Fri Dec 27 15:13:11 2013 -0800

----------------------------------------------------------------------
 .../apache/spark/deploy/client/DriverClientArguments.scala   | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c8c8b42a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
index 28bc549..0c84cc9 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
@@ -88,6 +88,12 @@ private[spark] class DriverClientArguments(args: Array[String]) {
    * Print usage and exit JVM with the given exit code.
    */
   def printUsageAndExit(exitCode: Int) {
+    // TODO: Document the submission approach here. It is:
+    //      1) Create an uber jar with your application and dependencies (excluding Spark)
+    //      2) You'll need to add this jar using addJar(X) inside of your spark context
+
+    // TODO: It wouldnt be too hard to allow users to submit their app and dependency jars
+    //       separately similar to in the YARN client.
     System.err.println(
       "usage: DriverClient [options] launch <active-master> <jar-url> <main-class> " +
         "[driver options]\n" +
@@ -95,7 +101,7 @@ private[spark] class DriverClientArguments(args: Array[String]) {
       "Options:\n" +
       "  -c CORES, --cores CORES                Number of cores to request \n" +
       "  -m MEMORY, --memory MEMORY             Megabytes of memory to request\n" +
-      "  -j JAVA_OPT, --java-option JAVA_OPT    Java option to pass to driver\n" +
+      "  -o JAVA_OPT, --java-option JAVA_OPT    JVM option to pass to driver\n" +
       "  -e K=V, --environment-variable K=V     Environment variable to pass to driver\n")
     System.exit(exitCode)
   }


[32/37] git commit: Show more helpful information in UI

Posted by pw...@apache.org.
Show more helpful information in UI


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

Branch: refs/heads/master
Commit: 3ec21f2eee0a322bcb5cce9678a816b012227a61
Parents: c78b381
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Jan 8 00:30:10 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Jan 8 00:30:10 2014 -0800

----------------------------------------------------------------------
 .../apache/spark/deploy/master/ui/IndexPage.scala   |  2 +-
 .../apache/spark/deploy/worker/DriverRunner.scala   | 16 ++++++++++------
 .../org/apache/spark/deploy/worker/Worker.scala     |  1 +
 .../apache/spark/deploy/worker/ui/IndexPage.scala   |  8 ++++++--
 4 files changed, 18 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3ec21f2e/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index a72d76b..db1dde5 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -162,7 +162,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
     <tr>
       <td>{driver.id} </td>
       <td>{driver.submitDate}</td>
-      <td>{driver.worker.map(w => w.id.toString).getOrElse("None")}</td>
+      <td>{driver.worker.map(w => <a href={w.webUiAddress}>{w.id.toString}</a>).getOrElse("None")}</td>
       <td>{driver.state}</td>
       <td sorttable_customkey={driver.desc.cores.toString}>
         {driver.desc.cores}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3ec21f2e/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index d13d7ef..ad70345 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -32,6 +32,7 @@ import org.apache.spark.Logging
 import org.apache.spark.deploy.{Command, DriverDescription}
 import org.apache.spark.deploy.DeployMessages.DriverStateChanged
 import org.apache.spark.deploy.master.DriverState
+import org.apache.spark.deploy.master.DriverState.DriverState
 
 /**
  * Manages the execution of one driver, including automatically restarting the driver on failure.
@@ -48,6 +49,10 @@ private[spark] class DriverRunner(
   @volatile var process: Option[Process] = None
   @volatile var killed = false
 
+  // Populated once finished
+  var finalState: Option[DriverState] = None
+  var finalException: Option[Exception] = None
+
   // Decoupled for testing
   private[deploy] def setClock(_clock: Clock) = clock = _clock
   private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper
@@ -62,8 +67,6 @@ private[spark] class DriverRunner(
   def start() = {
     new Thread("DriverRunner for " + driverId) {
       override def run() {
-        var exn: Option[Exception] = None
-
         try {
           val driverDir = createWorkingDirectory()
           val localJarFilename = downloadUserJar(driverDir)
@@ -79,15 +82,16 @@ private[spark] class DriverRunner(
           launchDriver(command, env, driverDir, driverDesc.supervise)
         }
         catch {
-          case e: Exception => exn = Some(e)
+          case e: Exception => finalException = Some(e)
         }
 
-        val finalState =
+        val state =
           if (killed) { DriverState.KILLED }
-          else if (exn.isDefined) { DriverState.FAILED }
+          else if (finalException.isDefined) { DriverState.FAILED }
           else { DriverState.FINISHED }
+        finalState = Some(state)
 
-        worker ! DriverStateChanged(driverId, finalState, exn)
+        worker ! DriverStateChanged(driverId, state, finalException)
       }
     }.start()
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3ec21f2e/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 4546e38..2a2b7a3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -31,6 +31,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException}
 import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.{DriverState, Master}
+import org.apache.spark.deploy.master.DriverState.DriverState
 import org.apache.spark.deploy.worker.ui.WorkerWebUI
 import org.apache.spark.metrics.MetricsSystem
 import org.apache.spark.util.{AkkaUtils, Utils}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3ec21f2e/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index 93c6ad4..0a7f56e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -18,7 +18,6 @@
 package org.apache.spark.deploy.worker.ui
 
 import scala.concurrent.Await
-import scala.concurrent.duration._
 import scala.xml.Node
 
 import akka.pattern.ask
@@ -27,6 +26,7 @@ import net.liftweb.json.JsonAST.JValue
 
 import org.apache.spark.deploy.JsonProtocol
 import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse}
+import org.apache.spark.deploy.master.DriverState
 import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner}
 import org.apache.spark.ui.UIUtils
 import org.apache.spark.util.Utils
@@ -52,7 +52,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
     val finishedExecutorTable =
       UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors)
 
-    val driverHeaders = Seq("DriverID", "Main Class", "Cores", "Memory", "Logs")
+    val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", "Memory", "Logs", "Notes")
     val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse
     val runningDriverTable = UIUtils.listingTable(driverHeaders, driverRow, runningDrivers)
     val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse
@@ -134,6 +134,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
     <tr>
       <td>{driver.driverId}</td>
       <td>{driver.driverDesc.command.mainClass}</td>
+      <td>{driver.finalState.getOrElse(DriverState.RUNNING)}</td>
       <td sorttable_customkey={driver.driverDesc.cores.toString}>
         {driver.driverDesc.cores.toString}
       </td>
@@ -144,6 +145,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
         <a href={s"logPage?driverId=${driver.driverId}&logType=stdout"}>stdout</a>
         <a href={s"logPage?driverId=${driver.driverId}&logType=stderr"}>stderr</a>
       </td>
+      <td>
+        {driver.finalException.getOrElse("")}
+      </td>
     </tr>
   }
 }


[23/37] git commit: Respect supervise option at Master

Posted by pw...@apache.org.
Respect supervise option at Master


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

Branch: refs/heads/master
Commit: 7a99702ce2fb04c4d76f0ce9f6df6608e0a5cce1
Parents: a872977
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sun Dec 29 12:12:50 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sun Dec 29 12:12:58 2013 -0800

----------------------------------------------------------------------
 .../org/apache/spark/deploy/master/Master.scala   | 18 +++++++++++++++---
 1 file changed, 15 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/7a99702c/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index b8655c7..29f20da 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -396,8 +396,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
 
     // Reschedule drivers which were not claimed by any workers
     drivers.filter(_.worker.isEmpty).foreach { d =>
-      logWarning(s"Driver ${d.id} was not found after master recovery, re-launching")
-      relaunchDriver(d)
+      logWarning(s"Driver ${d.id} was not found after master recovery")
+      if (d.desc.supervise) {
+        logWarning(s"Re-launching ${d.id}")
+        relaunchDriver(d)
+      } else {
+        removeDriver(d.id, DriverState.FAILED, None)
+        logWarning(s"Did not re-launch ${d.id} because it was not supervised")
+      }
     }
 
     state = RecoveryState.ALIVE
@@ -519,7 +525,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       exec.application.removeExecutor(exec)
     }
     for (driver <- worker.drivers.values) {
-      relaunchDriver(driver)
+      if (driver.desc.supervise) {
+        logInfo(s"Re-launching ${driver.id}")
+        relaunchDriver(driver)
+      } else {
+        logInfo(s"Not re-launching ${driver.id} because it was not supervised")
+        removeDriver(driver.id, DriverState.FAILED, None)
+      }
     }
     persistenceEngine.removeWorker(worker)
   }


[26/37] git commit: Fixes after merge

Posted by pw...@apache.org.
Fixes after merge


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

Branch: refs/heads/master
Commit: 6a3daead2d5c82136fefa2de9bced036d1ccb759
Parents: c0498f9
Author: Patrick Wendell <pw...@gmail.com>
Authored: Mon Jan 6 20:12:45 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Mon Jan 6 20:12:45 2014 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/deploy/client/DriverClient.scala   | 7 ++++---
 .../scala/org/apache/spark/deploy/worker/DriverWrapper.scala  | 5 +++--
 .../main/scala/org/apache/spark/deploy/worker/Worker.scala    | 2 +-
 3 files changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a3daead/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index e319e75..1cd5d99 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -23,7 +23,7 @@ import scala.concurrent._
 
 import akka.actor._
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
 import org.apache.spark.deploy.{Command, DriverDescription}
 import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.Master
@@ -59,11 +59,12 @@ object DriverClient extends Logging {
 
   def main(args: Array[String]) {
     val driverArgs = new DriverClientArguments(args)
+    val conf = new SparkConf()
 
     // TODO: See if we can initialize akka so return messages are sent back using the same TCP
     //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
     val (actorSystem, _) = AkkaUtils.createActorSystem(
-      "driverClient", Utils.localHostName(), 0)
+      "driverClient", Utils.localHostName(), 0, false, conf)
     val master = driverArgs.master
     val response = promise[(Boolean, String)]
     val driver: ActorRef = actorSystem.actorOf(Props(new DriverActor(driverArgs.master, response)))
@@ -95,7 +96,7 @@ object DriverClient extends Logging {
 
     val (success, message) =
       try {
-        Await.result(response.future, AkkaUtils.askTimeout)
+        Await.result(response.future, AkkaUtils.askTimeout(conf))
       } catch {
         case e: TimeoutException => (false, s"Master $master failed to respond in time")
       }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a3daead/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
index 2deb21a..1640d5f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -2,6 +2,7 @@ package org.apache.spark.deploy.worker
 
 import akka.actor._
 
+import org.apache.spark.SparkConf
 import org.apache.spark.util.{AkkaUtils, Utils}
 
 /**
@@ -12,7 +13,7 @@ object DriverWrapper {
     args.toList match {
       case workerUrl :: mainClass :: extraArgs =>
         val (actorSystem, _) = AkkaUtils.createActorSystem("Driver",
-          Utils.localHostName(), 0)
+          Utils.localHostName(), 0, false, new SparkConf())
         actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher")
 
         // Delegate to supplied main class
@@ -20,7 +21,7 @@ object DriverWrapper {
         val mainMethod = clazz.getMethod("main", classOf[Array[String]])
         mainMethod.invoke(null, extraArgs.toArray[String])
 
-        actorSystem.awaitTermination()
+        actorSystem.shutdown()
 
       case _ =>
         System.err.println("Usage: DriverWrapper <workerUrl> <driverMainClass> [options]")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6a3daead/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 2072f00..4546e38 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -335,7 +335,7 @@ private[spark] object Worker {
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port,
       conf = conf)
     actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
-      masterUrls, workDir, conf), name = actorName)
+      masterUrls, systemName, actorName,  workDir, conf), name = actorName)
     (actorSystem, boundPort)
   }
 


[06/37] git commit: Minor style clean-up

Posted by pw...@apache.org.
Minor style clean-up


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

Branch: refs/heads/master
Commit: c9c0f745afcf00c17fa073e4ca6dd9433400be95
Parents: b2b7514
Author: Patrick Wendell <pw...@gmail.com>
Authored: Wed Dec 25 00:54:34 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Dec 25 01:19:25 2013 -0800

----------------------------------------------------------------------
 .../spark/deploy/client/DriverClient.scala      |  2 ++
 .../org/apache/spark/deploy/master/Master.scala |  3 +--
 .../spark/deploy/master/ui/IndexPage.scala      | 22 ++++++++++----------
 .../spark/deploy/worker/DriverRunner.scala      |  4 ++--
 .../org/apache/spark/deploy/worker/Worker.scala |  3 ---
 .../spark/examples/DriverSubmissionTest.scala   |  1 +
 6 files changed, 17 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c9c0f745/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
index 9c0a626..28c851b 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClient.scala
@@ -101,6 +101,8 @@ object DriverClient {
   def main(args: Array[String]) {
     val driverArgs = new DriverClientArguments(args)
 
+    // TODO: See if we can initialize akka so return messages are sent back using the same TCP
+    //       flow. Else, this (sadly) requires the DriverClient be routable from the Master.
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(
       "driverClient", Utils.localHostName(), 0)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c9c0f745/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index f5d6fda..0528ef4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -185,7 +185,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
         schedule()
 
         // TODO: It might be good to instead have the submission client poll the master to determine
-        //       the current status of the driver. Since we may already want to expose this.
+        //       the current status of the driver. For now it's simply "fire and forget".
 
         sender ! SubmitDriverResponse(true, "Driver successfully submitted")
       }
@@ -611,7 +611,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }
   }
 
-  /** Generate a new driver ID given a driver's submission date */
   def newDriverId(submitDate: Date): String = {
     val appId = "driver-%s-%04d".format(DATE_FORMAT.format(submitDate), nextDriverNumber)
     nextDriverNumber += 1

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c9c0f745/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index 6a99d7a..3c6fca3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -106,20 +106,20 @@ private[spark] class IndexPage(parent: MasterWebUI) {
           </div>
         </div>
 
-          <div class="row-fluid">
-            <div class="span12">
-              <h4> Active Drivers </h4>
+        <div class="row-fluid">
+          <div class="span12">
+            <h4> Active Drivers </h4>
 
-              {activeDriversTable}
-            </div>
+            {activeDriversTable}
           </div>
+        </div>
 
-          <div class="row-fluid">
-            <div class="span12">
-              <h4> Completed Drivers </h4>
-              {completedDriversTable}
-            </div>
-          </div>;
+        <div class="row-fluid">
+          <div class="span12">
+            <h4> Completed Drivers </h4>
+            {completedDriversTable}
+          </div>
+        </div>;
     UIUtils.basicSparkPage(content, "Spark Master at " + state.uri)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c9c0f745/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index b030d60..28d4297 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -32,7 +32,7 @@ import org.apache.spark.deploy.master.DriverState
 import org.apache.spark.util.Utils
 
 /**
- * Manages the execution of one driver process.
+ * Manages the execution of one driver, including automatically restarting the driver on failure.
  */
 private[spark] class DriverRunner(
     val driverId: String,
@@ -133,7 +133,7 @@ private[spark] class DriverRunner(
     localJarFilename
   }
 
-  /** Continue launching the supplied command until it exits zero. */
+  /** Continue launching the supplied command until it exits zero or is killed. */
   def runCommandWithRetry(command: Seq[String], envVars: Seq[(String, String)], baseDir: File) = {
     // Time to wait between submission retries.
     var waitSeconds = 1

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c9c0f745/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index b6a84fc..42c28cf 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -254,17 +254,14 @@ private[spark] class Worker(
 
     case KillDriver(driverId) => {
       logInfo(s"Asked to kill driver $driverId")
-
       drivers.find(_._1 == driverId) match {
         case Some((id, runner)) =>
           runner.kill()
         case None =>
           logError(s"Asked to kill unknown driver $driverId")
       }
-
     }
 
-
     case DriverStateChanged(driverId, state, exception) => {
       state match {
         case DriverState.FAILED =>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c9c0f745/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
index 9055ce7..65251e9 100644
--- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
@@ -39,6 +39,7 @@ object DriverSubmissionTest {
     properties.filter{case (k, v) => k.toString.contains("spark.test")}.foreach(println)
 
     for (i <- 1 until numSecondsToSleep) {
+      println(s"Alive for $i out of $numSecondsToSleep seconds")
       Thread.sleep(1000)
     }
   }


[14/37] git commit: Addressing smaller changes from Aaron's review

Posted by pw...@apache.org.
Addressing smaller changes from Aaron's review


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

Branch: refs/heads/master
Commit: c23d640516e05a32f1380cdd3d35bf948c92cd60
Parents: da20270
Author: Patrick Wendell <pw...@gmail.com>
Authored: Thu Dec 26 12:37:01 2013 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Thu Dec 26 12:38:39 2013 -0800

----------------------------------------------------------------------
 .../org/apache/spark/deploy/DeployMessage.scala |  5 ++--
 .../deploy/client/DriverClientArguments.scala   |  2 +-
 .../apache/spark/deploy/master/DriverInfo.scala |  1 -
 .../org/apache/spark/deploy/master/Master.scala | 30 +++++++++++++-------
 .../apache/spark/deploy/master/WorkerInfo.scala |  4 +--
 .../spark/deploy/master/ui/IndexPage.scala      |  4 +--
 .../org/apache/spark/deploy/worker/Worker.scala | 12 ++++----
 7 files changed, 31 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c23d6405/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 82bb33a..7bfc377 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -83,8 +83,7 @@ private[deploy] object DeployMessages {
       sparkHome: String)
     extends DeployMessage
 
-  case class LaunchDriver(driverId: String, driverDesc: DriverDescription)
-    extends DeployMessage
+  case class LaunchDriver(driverId: String, driverDesc: DriverDescription) extends DeployMessage
 
   case class KillDriver(driverId: String) extends DeployMessage
 
@@ -134,8 +133,8 @@ private[deploy] object DeployMessages {
   // Master to MasterWebUI
 
   case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo],
-    activeDrivers: Array[DriverInfo], completedDrivers: Array[DriverInfo],
     activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo],
+    activeDrivers: Array[DriverInfo], completedDrivers: Array[DriverInfo],
     status: MasterState) {
 
     Utils.checkHost(host, "Required hostname")

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c23d6405/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
index 60e6549..28bc549 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/DriverClientArguments.scala
@@ -89,7 +89,7 @@ private[spark] class DriverClientArguments(args: Array[String]) {
    */
   def printUsageAndExit(exitCode: Int) {
     System.err.println(
-      "usage: DriverClient launch [options] <active-master> <jar-url> <main-class> " +
+      "usage: DriverClient [options] launch <active-master> <jar-url> <main-class> " +
         "[driver options]\n" +
       "usage: DriverClient kill <active-master> <driver-id>\n\n" +
       "Options:\n" +

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c23d6405/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
index 052c474..3337793 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala
@@ -33,5 +33,4 @@ private[spark] class DriverInfo(
   @transient var exception: Option[Exception] = None
   /* Most recent worker assigned to this driver */
   @transient var worker: Option[WorkerInfo] = None
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c23d6405/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 0528ef4..7f9ad8a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -19,11 +19,11 @@ package org.apache.spark.deploy.master
 
 import java.text.SimpleDateFormat
 import java.util.Date
-import java.util.concurrent.TimeUnit
 
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 import scala.concurrent.Await
 import scala.concurrent.duration._
+import scala.util.Random
 
 import akka.actor._
 import akka.pattern.ask
@@ -174,8 +174,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }
 
     case RequestSubmitDriver(description) => {
-      if (state == RecoveryState.STANDBY) {
-        sender ! SubmitDriverResponse(false, "Standby master cannot accept driver submission")
+      if (state != RecoveryState.ALIVE) {
+        val msg = s"Can only accept driver submissions in ALIVE state. Current state: $state."
+        sender ! SubmitDriverResponse(false, msg)
       } else {
         logInfo("Driver submitted " + description.mainClass)
         val driver = createDriver(description)
@@ -192,14 +193,18 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }
 
     case RequestKillDriver(driverId) => {
-      if (state == RecoveryState.STANDBY) {
-        sender ! KillDriverResponse(false, "Standby master cannot kill drivers")
+      if (state != RecoveryState.ALIVE) {
+        val msg = s"Can only kill drivers in ALIVE state. Current state: $state."
+        sender ! KillDriverResponse(false, msg)
       } else {
         logInfo("Asked to kill driver " + driverId)
         val driver = drivers.find(_.id == driverId)
         driver match {
           case Some(d) =>
-            if (waitingDrivers.contains(d)) { waitingDrivers -= d }
+            if (waitingDrivers.contains(d)) {
+              waitingDrivers -= d
+              self ! DriverStateChanged(driverId, DriverState.KILLED, None)
+            }
             else {
               // We just notify the worker to kill the driver here. The final bookkeeping occurs
               // on the return path when the worker submits a state change back to the master
@@ -208,6 +213,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
                 w.actor ! KillDriver(driverId)
               }
             }
+            // TODO: It would be nice for this to be a synchronous response
             val msg = s"Kill request for $driverId submitted"
             logInfo(msg)
             sender ! KillDriverResponse(true, msg)
@@ -338,8 +344,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
     }
 
     case RequestMasterState => {
-      sender ! MasterStateResponse(host, port, workers.toArray, drivers.toArray,
-        completedDrivers.toArray ,apps.toArray, completedApps.toArray, state)
+      sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray,
+        drivers.toArray, completedDrivers.toArray, state)
     }
 
     case CheckForWorkerTimeOut => {
@@ -423,10 +429,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
    */
   def schedule() {
     if (state != RecoveryState.ALIVE) { return }
+
     // First schedule drivers, they take strict precedence over applications
-    for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) {
-      for (driver <- Seq(waitingDrivers: _*)) {
-        if (worker.memoryFree > driver.desc.mem && worker.coresFree > driver.desc.cores) {
+    val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers
+    for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) {
+      for (driver <- waitingDrivers) {
+        if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) {
           launchDriver(worker, driver)
           waitingDrivers -= driver
         }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c23d6405/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index 28cd463..c5fa9cf 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -37,8 +37,8 @@ private[spark] class WorkerInfo(
   Utils.checkHost(host, "Expected hostname")
   assert (port > 0)
 
-  @transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // fullId => info
-  @transient var drivers: mutable.HashMap[String, DriverInfo] = _
+  @transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // executorId => info
+  @transient var drivers: mutable.HashMap[String, DriverInfo] = _ // driverId => info
   @transient var state: WorkerState.Value = _
   @transient var coresUsed: Int = _
   @transient var memoryUsed: Int = _

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c23d6405/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index 3c6fca3..951fc67 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -94,7 +94,6 @@ private[spark] class IndexPage(parent: MasterWebUI) {
         <div class="row-fluid">
           <div class="span12">
             <h4> Running Applications </h4>
-
             {activeAppsTable}
           </div>
         </div>
@@ -109,7 +108,6 @@ private[spark] class IndexPage(parent: MasterWebUI) {
         <div class="row-fluid">
           <div class="span12">
             <h4> Active Drivers </h4>
-
             {activeDriversTable}
           </div>
         </div>
@@ -167,7 +165,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
       <td>{driver.worker.map(w => w.id.toString).getOrElse("None")}</td>
       <td>{driver.state}</td>
       <td sorttable_customkey={driver.desc.cores.toString}>
-        {driver.desc.cores.toString}
+        {driver.desc.cores}
       </td>
       <td sorttable_customkey={driver.desc.mem.toString}>
         {Utils.megabytesToString(driver.desc.mem.toLong)}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c23d6405/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 42c28cf..21ec881 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -222,8 +222,8 @@ private[spark] class Worker(
         logInfo("Executor " + fullId + " finished with state " + state +
           message.map(" message " + _).getOrElse("") +
           exitStatus.map(" exitStatus " + _).getOrElse(""))
-        finishedExecutors(fullId) = executor
         executors -= fullId
+        finishedExecutors(fullId) = executor
         coresUsed -= executor.cores
         memoryUsed -= executor.memory
       }
@@ -248,8 +248,8 @@ private[spark] class Worker(
       drivers(driverId) = driver
       driver.start()
 
-      coresUsed += 1
-      memoryUsed += memory
+      coresUsed += driverDesc.cores
+      memoryUsed += driverDesc.mem
     }
 
     case KillDriver(driverId) => {
@@ -269,16 +269,16 @@ private[spark] class Worker(
         case DriverState.FINISHED =>
           logInfo(s"Driver $driverId exited successfully")
         case DriverState.KILLED =>
-          logInfo(s"Driver $driverId was killed")
+          logInfo(s"Driver $driverId was killed by user")
       }
       masterLock.synchronized {
         master ! DriverStateChanged(driverId, state, exception)
       }
       val driver = drivers(driverId)
-      memoryUsed -= driver.driverDesc.mem
-      coresUsed -= driver.driverDesc.cores
       drivers -= driverId
       finishedDrivers(driverId) = driver
+      memoryUsed -= driver.driverDesc.mem
+      coresUsed -= driver.driverDesc.cores
     }
 
     case x: DisassociatedEvent if x.remoteAddress == masterAddress =>