-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-1112, 2156] Bootstrap to fetch the driver's Spark properties. #1132
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 3 commits
4ab696a
7947c18
46d332d
68e1dfb
77ff32d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,9 +18,14 @@ | |
| package org.apache.spark.executor | ||
|
|
||
| import java.nio.ByteBuffer | ||
| import java.util.concurrent.TimeUnit | ||
|
|
||
| import scala.concurrent.Await | ||
|
|
||
| import akka.actor._ | ||
| import akka.remote._ | ||
| import akka.pattern.Patterns | ||
| import akka.util.Timeout | ||
|
|
||
| import org.apache.spark.{SparkEnv, Logging, SecurityManager, SparkConf} | ||
| import org.apache.spark.TaskState.TaskState | ||
|
|
@@ -101,26 +106,33 @@ private[spark] object CoarseGrainedExecutorBackend { | |
| workerUrl: Option[String]) { | ||
|
|
||
| SparkHadoopUtil.get.runAsSparkUser { () => | ||
| // Debug code | ||
| Utils.checkHost(hostname) | ||
|
|
||
| val conf = new SparkConf | ||
| // Create a new ActorSystem to run the backend, because we can't create a | ||
| // SparkEnv / Executor before getting started with all our system properties, etc | ||
| val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, | ||
| conf, new SecurityManager(conf)) | ||
| // set it | ||
| val sparkHostPort = hostname + ":" + boundPort | ||
| actorSystem.actorOf( | ||
| Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, | ||
| sparkHostPort, cores), | ||
| name = "Executor") | ||
| workerUrl.foreach { | ||
| url => | ||
| actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") | ||
| } | ||
| actorSystem.awaitTermination() | ||
|
|
||
| // Debug code | ||
| Utils.checkHost(hostname) | ||
|
|
||
| // Bootstrap to fetch the driver's Spark properties. | ||
| val executorConf = new SparkConf | ||
| val (fetcher, _) = AkkaUtils.createActorSystem( | ||
| "driverPropsFetcher", hostname, 0, executorConf, new SecurityManager(executorConf)) | ||
| val driver = fetcher.actorSelection(driverUrl) | ||
| val timeout = new Timeout(5, TimeUnit.MINUTES) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Any particular reason you chose 5 minutes over the AkkaUtils.askTimeout?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No, I didn't see |
||
| val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) | ||
| val props = Await.result(fut, timeout.duration).asInstanceOf[Seq[(String, String)]] | ||
| fetcher.shutdown() | ||
|
|
||
| // Create a new ActorSystem to run the backend, because we can't create a | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This comment is no longer technically true -- I think we do have all the information to start a proper SparkEnv, whose actor system we could use instead. Also, RegisteredExecutor() no longer needs t otake these properties.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It requires changing |
||
| // SparkEnv / Executor before getting started with all our system properties, etc | ||
| val driverConf = new SparkConf().setAll(props) | ||
| val (actorSystem, boundPort) = AkkaUtils.createActorSystem( | ||
| "sparkExecutor", hostname, 0, driverConf, new SecurityManager(driverConf)) | ||
| // set it | ||
| val sparkHostPort = hostname + ":" + boundPort | ||
| actorSystem.actorOf( | ||
| Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), | ||
| name = "Executor") | ||
| workerUrl.foreach { url => | ||
| actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") | ||
| } | ||
| actorSystem.awaitTermination() | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,11 +22,14 @@ import java.nio.ByteBuffer | |
| import org.apache.spark.TaskState.TaskState | ||
| import org.apache.spark.scheduler.TaskDescription | ||
| import org.apache.spark.util.{SerializableBuffer, Utils} | ||
| import org.apache.spark.SparkConf | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: import ordering (this should come first) |
||
|
|
||
| private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable | ||
|
|
||
| private[spark] object CoarseGrainedClusterMessages { | ||
|
|
||
| case object RetrieveSparkProps extends CoarseGrainedClusterMessage | ||
|
|
||
| // Driver to executors | ||
| case class LaunchTask(data: SerializableBuffer) extends CoarseGrainedClusterMessage | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: alphabetize imports