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 be8693ec5435eb0c9f8d18f67715c31493fec28b..3953a3e1786ecb85bfb43267c7aa6aca5a2fbf9e 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) { 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 26f980760df10fe5374ea6a0c326467b6353218b..0545ad185f418cb2d05bfc65081e5d67b3fc8a17 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 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 991b22d9f897fc3b69b33307a36a42d0d8e44112..98c57ca0b0b4bd0b55c79192b677ea1007ac84df 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) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9a3cbbe7d2652c855bba987f3a3aba6f2a5d5788..26e6a8326cbfc95d498901c049a3c43275600bd1 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",