Some fixes for previous master merge commits
This commit is contained in:
parent
a60620b76a
commit
d4cd32330e
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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",
|
||||
|
|
Loading…
Reference in a new issue