Some fixes for previous master merge commits

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

Branch: refs/heads/master
Commit: d4cd32330e1e4ac83b38bc922a9d3fd85f85f606
Parents: a60620b
Author: Raymond Liu <[email protected]>
Authored: Thu Nov 14 13:57:02 2013 +0800
Committer: Raymond Liu <[email protected]>
Committed: Fri Nov 15 10:22:31 2013 +0800

----------------------------------------------------------------------
 core/src/main/scala/org/apache/spark/deploy/client/Client.scala | 2 +-
 core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 3 ++-
 core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 4 +---
 project/SparkBuild.scala                                        | 1 +
 4 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d4cd3233/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
index be8693e..3953a3e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
@@ -21,7 +21,6 @@ import java.util.concurrent.TimeoutException
 
 import scala.concurrent.duration._
 import scala.concurrent.Await
-import scala.concurrent.ExecutionContext.Implicits.global
 
 import akka.actor._
 import akka.actor.Terminated
@@ -84,6 +83,7 @@ private[spark] class Client(
     def registerWithMaster() {
       tryRegisterAllMasters()
 
+      import context.dispatcher
       var retries = 0
       lazy val retryTimer: Cancellable =
         context.system.scheduler.schedule(REGISTRATION_TIMEOUT, 
REGISTRATION_TIMEOUT) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d4cd3233/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 26f9807..0545ad1 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
@@ -24,7 +24,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, 
HashSet}
 import scala.concurrent.Await
 import scala.concurrent.duration._
 import scala.concurrent.duration.{ Duration, FiniteDuration }
-import scala.concurrent.ExecutionContext.Implicits.global
 
 import akka.actor._
 import akka.pattern.ask
@@ -58,6 +57,8 @@ import java.util.concurrent.TimeUnit
 
 
 private[spark] class Master(host: String, port: Int, webUiPort: Int) extends 
Actor with Logging {
+  import context.dispatcher
+
   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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d4cd3233/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 991b22d..98c57ca 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
@@ -23,7 +23,6 @@ import java.io.File
 
 import scala.collection.mutable.HashMap
 import scala.concurrent.duration._
-import scala.concurrent.ExecutionContext.Implicits.global
 
 import akka.actor._
 import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, 
DisassociatedEvent}
@@ -61,6 +60,7 @@ private[spark] class Worker(
     masterUrls: Array[String],
     workDirPath: String = null)
   extends Actor with Logging {
+  import context.dispatcher
 
   Utils.checkHost(host, "Expected hostname")
   assert (port > 0)
@@ -175,8 +175,6 @@ private[spark] class Worker(
     retryTimer // start timer
   }
 
-  import context.dispatcher
-
   override def receive = {
     case RegisteredWorker(masterUrl, masterWebUiUrl) =>
       logInfo("Successfully registered with master " + masterUrl)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d4cd3233/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 9a3cbbe..26e6a83 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -227,6 +227,7 @@ object SparkBuild extends Build {
         "org.apache.hadoop"        % "hadoop-client"    % hadoopVersion 
excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
         "org.apache.avro"          % "avro"             % "1.7.4",
         "org.apache.avro"          % "avro-ipc"         % "1.7.4" 
excludeAll(excludeNetty),
+        "org.apache.zookeeper"     % "zookeeper"        % "3.4.5" 
excludeAll(excludeNetty),
         "com.codahale.metrics"     % "metrics-core"     % "3.0.0",
         "com.codahale.metrics"     % "metrics-jvm"      % "3.0.0",
         "com.codahale.metrics"     % "metrics-json"     % "3.0.0",

Reply via email to