18
18
package org .apache .spark .executor
19
19
20
20
import java .nio .ByteBuffer
21
- import java .util .concurrent .TimeUnit
22
21
23
22
import scala .concurrent .Await
24
23
25
24
import akka .actor ._
26
25
import akka .remote ._
27
26
import akka .pattern .Patterns
28
- import akka .util .Timeout
29
27
30
28
import org .apache .spark .{SparkEnv , Logging , SecurityManager , SparkConf }
31
29
import org .apache .spark .TaskState .TaskState
@@ -39,10 +37,8 @@ private[spark] class CoarseGrainedExecutorBackend(
39
37
driverUrl : String ,
40
38
executorId : String ,
41
39
hostPort : String ,
42
- cores : Int )
43
- extends Actor
44
- with ExecutorBackend
45
- with Logging {
40
+ cores : Int ,
41
+ sparkProperties : Seq [(String , String )]) extends Actor with ExecutorBackend with Logging {
46
42
47
43
Utils .checkHostPort(hostPort, " Expected hostport" )
48
44
@@ -57,7 +53,7 @@ private[spark] class CoarseGrainedExecutorBackend(
57
53
}
58
54
59
55
override def receive = {
60
- case RegisteredExecutor (sparkProperties) =>
56
+ case RegisteredExecutor =>
61
57
logInfo(" Successfully registered with driver" )
62
58
// Make this host instead of hostPort ?
63
59
executor = new Executor (executorId, Utils .parseHostPort(hostPort)._1, sparkProperties,
@@ -114,20 +110,20 @@ private[spark] object CoarseGrainedExecutorBackend {
114
110
val (fetcher, _) = AkkaUtils .createActorSystem(
115
111
" driverPropsFetcher" , hostname, 0 , executorConf, new SecurityManager (executorConf))
116
112
val driver = fetcher.actorSelection(driverUrl)
117
- val timeout = new Timeout ( 5 , TimeUnit . MINUTES )
113
+ val timeout = AkkaUtils .askTimeout(executorConf )
118
114
val fut = Patterns .ask(driver, RetrieveSparkProps , timeout)
119
- val props = Await .result(fut, timeout.duration ).asInstanceOf [Seq [(String , String )]]
115
+ val props = Await .result(fut, timeout).asInstanceOf [Seq [(String , String )]]
120
116
fetcher.shutdown()
121
117
122
- // Create a new ActorSystem to run the backend, because we can't create a
123
- // SparkEnv / Executor before getting started with all our system properties, etc
118
+ // Create a new ActorSystem using driver's Spark properties to run the backend.
124
119
val driverConf = new SparkConf ().setAll(props)
125
120
val (actorSystem, boundPort) = AkkaUtils .createActorSystem(
126
121
" sparkExecutor" , hostname, 0 , driverConf, new SecurityManager (driverConf))
127
122
// set it
128
123
val sparkHostPort = hostname + " :" + boundPort
129
124
actorSystem.actorOf(
130
- Props (classOf [CoarseGrainedExecutorBackend ], driverUrl, executorId, sparkHostPort, cores),
125
+ Props (classOf [CoarseGrainedExecutorBackend ],
126
+ driverUrl, executorId, sparkHostPort, cores, props),
131
127
name = " Executor" )
132
128
workerUrl.foreach { url =>
133
129
actorSystem.actorOf(Props (classOf [WorkerWatcher ], url), name = " WorkerWatcher" )
0 commit comments