@@ -19,6 +19,7 @@ package org.apache.spark.scheduler
19
19
20
20
import java .io .NotSerializableException
21
21
import java .util .Properties
22
+ import java .util .concurrent .{TimeUnit , Executors }
22
23
import java .util .concurrent .atomic .AtomicInteger
23
24
24
25
import scala .collection .mutable .{ArrayBuffer , HashMap , HashSet , Map , Stack }
@@ -28,8 +29,6 @@ import scala.language.postfixOps
28
29
import scala .reflect .ClassTag
29
30
import scala .util .control .NonFatal
30
31
31
- import akka .actor ._
32
- import akka .actor .SupervisorStrategy .Stop
33
32
import akka .pattern .ask
34
33
import akka .util .Timeout
35
34
@@ -39,7 +38,7 @@ import org.apache.spark.executor.TaskMetrics
39
38
import org .apache .spark .partial .{ApproximateActionListener , ApproximateEvaluator , PartialResult }
40
39
import org .apache .spark .rdd .RDD
41
40
import org .apache .spark .storage ._
42
- import org .apache .spark .util .{CallSite , SystemClock , Clock , Utils }
41
+ import org .apache .spark .util .{CallSite , EventLoop , SystemClock , Clock , Utils }
43
42
import org .apache .spark .storage .BlockManagerMessages .BlockManagerHeartbeat
44
43
45
44
/**
@@ -67,8 +66,6 @@ class DAGScheduler(
67
66
clock : Clock = SystemClock )
68
67
extends Logging {
69
68
70
- import DAGScheduler ._
71
-
72
69
def this (sc : SparkContext , taskScheduler : TaskScheduler ) = {
73
70
this (
74
71
sc,
@@ -112,42 +109,31 @@ class DAGScheduler(
112
109
// stray messages to detect.
113
110
private val failedEpoch = new HashMap [String , Long ]
114
111
115
- private val dagSchedulerActorSupervisor =
116
- env.actorSystem.actorOf(Props (new DAGSchedulerActorSupervisor (this )))
117
-
118
112
// A closure serializer that we reuse.
119
113
// This is only safe because DAGScheduler runs in a single thread.
120
114
private val closureSerializer = SparkEnv .get.closureSerializer.newInstance()
121
115
122
- private [scheduler] var eventProcessActor : ActorRef = _
123
116
124
117
/** If enabled, we may run certain actions like take() and first() locally. */
125
118
private val localExecutionEnabled = sc.getConf.getBoolean(" spark.localExecution.enabled" , false )
126
119
127
120
/** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */
128
121
private val disallowStageRetryForTest = sc.getConf.getBoolean(" spark.test.noStageRetry" , false )
129
122
130
- private def initializeEventProcessActor () {
131
- // blocking the thread until supervisor is started, which ensures eventProcessActor is
132
- // not null before any job is submitted
133
- implicit val timeout = Timeout (30 seconds)
134
- val initEventActorReply =
135
- dagSchedulerActorSupervisor ? Props (new DAGSchedulerEventProcessActor (this ))
136
- eventProcessActor = Await .result(initEventActorReply, timeout.duration).
137
- asInstanceOf [ActorRef ]
138
- }
123
+ private val messageScheduler =
124
+ Executors .newScheduledThreadPool(1 , Utils .namedThreadFactory(" dag-scheduler-message" ))
139
125
140
- initializeEventProcessActor( )
126
+ private [scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop ( this )
141
127
taskScheduler.setDAGScheduler(this )
142
128
143
129
// Called by TaskScheduler to report task's starting.
144
130
def taskStarted (task : Task [_], taskInfo : TaskInfo ) {
145
- eventProcessActor ! BeginEvent (task, taskInfo)
131
+ eventProcessLoop.post( BeginEvent (task, taskInfo) )
146
132
}
147
133
148
134
// Called to report that a task has completed and results are being fetched remotely.
149
135
def taskGettingResult (taskInfo : TaskInfo ) {
150
- eventProcessActor ! GettingResultEvent (taskInfo)
136
+ eventProcessLoop.post( GettingResultEvent (taskInfo) )
151
137
}
152
138
153
139
// Called by TaskScheduler to report task completions or failures.
@@ -158,7 +144,8 @@ class DAGScheduler(
158
144
accumUpdates : Map [Long , Any ],
159
145
taskInfo : TaskInfo ,
160
146
taskMetrics : TaskMetrics ) {
161
- eventProcessActor ! CompletionEvent (task, reason, result, accumUpdates, taskInfo, taskMetrics)
147
+ eventProcessLoop.post(
148
+ CompletionEvent (task, reason, result, accumUpdates, taskInfo, taskMetrics))
162
149
}
163
150
164
151
/**
@@ -180,18 +167,18 @@ class DAGScheduler(
180
167
181
168
// Called by TaskScheduler when an executor fails.
182
169
def executorLost (execId : String ) {
183
- eventProcessActor ! ExecutorLost (execId)
170
+ eventProcessLoop.post( ExecutorLost (execId) )
184
171
}
185
172
186
173
// Called by TaskScheduler when a host is added
187
174
def executorAdded (execId : String , host : String ) {
188
- eventProcessActor ! ExecutorAdded (execId, host)
175
+ eventProcessLoop.post( ExecutorAdded (execId, host) )
189
176
}
190
177
191
178
// Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or
192
179
// cancellation of the job itself.
193
180
def taskSetFailed (taskSet : TaskSet , reason : String ) {
194
- eventProcessActor ! TaskSetFailed (taskSet, reason)
181
+ eventProcessLoop.post( TaskSetFailed (taskSet, reason) )
195
182
}
196
183
197
184
private def getCacheLocs (rdd : RDD [_]): Array [Seq [TaskLocation ]] = {
@@ -496,8 +483,8 @@ class DAGScheduler(
496
483
assert(partitions.size > 0 )
497
484
val func2 = func.asInstanceOf [(TaskContext , Iterator [_]) => _]
498
485
val waiter = new JobWaiter (this , jobId, partitions.size, resultHandler)
499
- eventProcessActor ! JobSubmitted (
500
- jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties)
486
+ eventProcessLoop.post( JobSubmitted (
487
+ jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties))
501
488
waiter
502
489
}
503
490
@@ -537,8 +524,8 @@ class DAGScheduler(
537
524
val func2 = func.asInstanceOf [(TaskContext , Iterator [_]) => _]
538
525
val partitions = (0 until rdd.partitions.size).toArray
539
526
val jobId = nextJobId.getAndIncrement()
540
- eventProcessActor ! JobSubmitted (
541
- jobId, rdd, func2, partitions, allowLocal = false , callSite, listener, properties)
527
+ eventProcessLoop.post( JobSubmitted (
528
+ jobId, rdd, func2, partitions, allowLocal = false , callSite, listener, properties))
542
529
listener.awaitResult() // Will throw an exception if the job fails
543
530
}
544
531
@@ -547,19 +534,19 @@ class DAGScheduler(
547
534
*/
548
535
def cancelJob (jobId : Int ) {
549
536
logInfo(" Asked to cancel job " + jobId)
550
- eventProcessActor ! JobCancelled (jobId)
537
+ eventProcessLoop.post( JobCancelled (jobId) )
551
538
}
552
539
553
540
def cancelJobGroup (groupId : String ) {
554
541
logInfo(" Asked to cancel job group " + groupId)
555
- eventProcessActor ! JobGroupCancelled (groupId)
542
+ eventProcessLoop.post( JobGroupCancelled (groupId) )
556
543
}
557
544
558
545
/**
559
546
* Cancel all jobs that are running or waiting in the queue.
560
547
*/
561
548
def cancelAllJobs () {
562
- eventProcessActor ! AllJobsCancelled
549
+ eventProcessLoop.post( AllJobsCancelled )
563
550
}
564
551
565
552
private [scheduler] def doCancelAllJobs () {
@@ -575,7 +562,7 @@ class DAGScheduler(
575
562
* Cancel all jobs associated with a running or scheduled stage.
576
563
*/
577
564
def cancelStage (stageId : Int ) {
578
- eventProcessActor ! StageCancelled (stageId)
565
+ eventProcessLoop.post( StageCancelled (stageId) )
579
566
}
580
567
581
568
/**
@@ -1059,16 +1046,16 @@ class DAGScheduler(
1059
1046
1060
1047
if (disallowStageRetryForTest) {
1061
1048
abortStage(failedStage, " Fetch failure will not retry stage due to testing config" )
1062
- } else if (failedStages.isEmpty && eventProcessActor != null ) {
1049
+ } else if (failedStages.isEmpty && eventProcessLoop != null ) {
1063
1050
// Don't schedule an event to resubmit failed stages if failed isn't empty, because
1064
- // in that case the event will already have been scheduled. eventProcessActor may be
1051
+ // in that case the event will already have been scheduled. eventProcessLoop may be
1065
1052
// null during unit tests.
1066
1053
// TODO: Cancel running tasks in the stage
1067
- import env .actorSystem .dispatcher
1068
1054
logInfo(s " Resubmitting $mapStage ( ${mapStage.name}) and " +
1069
1055
s " $failedStage ( ${failedStage.name}) due to fetch failure " )
1070
- env.actorSystem.scheduler.scheduleOnce(
1071
- RESUBMIT_TIMEOUT , eventProcessActor, ResubmitFailedStages )
1056
+ messageScheduler.schedule(new Runnable {
1057
+ override def run (): Unit = eventProcessLoop.post(ResubmitFailedStages )
1058
+ }, DAGScheduler .RESUBMIT_TIMEOUT , TimeUnit .MILLISECONDS )
1072
1059
}
1073
1060
failedStages += failedStage
1074
1061
failedStages += mapStage
@@ -1326,40 +1313,21 @@ class DAGScheduler(
1326
1313
1327
1314
def stop () {
1328
1315
logInfo(" Stopping DAGScheduler" )
1329
- dagSchedulerActorSupervisor ! PoisonPill
1316
+ eventProcessLoop.stop()
1330
1317
taskScheduler.stop()
1331
1318
}
1332
- }
1333
1319
1334
- private [scheduler] class DAGSchedulerActorSupervisor (dagScheduler : DAGScheduler )
1335
- extends Actor with Logging {
1336
-
1337
- override val supervisorStrategy =
1338
- OneForOneStrategy () {
1339
- case x : Exception =>
1340
- logError(" eventProcesserActor failed; shutting down SparkContext" , x)
1341
- try {
1342
- dagScheduler.doCancelAllJobs()
1343
- } catch {
1344
- case t : Throwable => logError(" DAGScheduler failed to cancel all jobs." , t)
1345
- }
1346
- dagScheduler.sc.stop()
1347
- Stop
1348
- }
1349
-
1350
- def receive = {
1351
- case p : Props => sender ! context.actorOf(p)
1352
- case _ => logWarning(" received unknown message in DAGSchedulerActorSupervisor" )
1353
- }
1320
+ // Start the event thread at the end of the constructor
1321
+ eventProcessLoop.start()
1354
1322
}
1355
1323
1356
- private [scheduler] class DAGSchedulerEventProcessActor (dagScheduler : DAGScheduler )
1357
- extends Actor with Logging {
1324
+ private [scheduler] class DAGSchedulerEventProcessLoop (dagScheduler : DAGScheduler )
1325
+ extends EventLoop [ DAGSchedulerEvent ]( " dag-scheduler-event-loop " ) with Logging {
1358
1326
1359
1327
/**
1360
1328
* The main event loop of the DAG scheduler.
1361
1329
*/
1362
- def receive = {
1330
+ override def onReceive ( event : DAGSchedulerEvent ) : Unit = event match {
1363
1331
case JobSubmitted (jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) =>
1364
1332
dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite,
1365
1333
listener, properties)
@@ -1398,7 +1366,17 @@ private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGSchedule
1398
1366
dagScheduler.resubmitFailedStages()
1399
1367
}
1400
1368
1401
- override def postStop () {
1369
+ override def onError (e : Throwable ): Unit = {
1370
+ logError(" DAGSchedulerEventProcessLoop failed; shutting down SparkContext" , e)
1371
+ try {
1372
+ dagScheduler.doCancelAllJobs()
1373
+ } catch {
1374
+ case t : Throwable => logError(" DAGScheduler failed to cancel all jobs." , t)
1375
+ }
1376
+ dagScheduler.sc.stop()
1377
+ }
1378
+
1379
+ override def onStop () {
1402
1380
// Cancel any active jobs in postStop hook
1403
1381
dagScheduler.cleanUpAfterSchedulerStop()
1404
1382
}
@@ -1408,9 +1386,5 @@ private[spark] object DAGScheduler {
1408
1386
// The time, in millis, to wait for fetch failure events to stop coming in after one is detected;
1409
1387
// this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one
1410
1388
// as more failure events come in
1411
- val RESUBMIT_TIMEOUT = 200 .milliseconds
1412
-
1413
- // The time, in millis, to wake up between polls of the completion queue in order to potentially
1414
- // resubmit failed stages
1415
- val POLL_TIMEOUT = 10L
1389
+ val RESUBMIT_TIMEOUT = 200
1416
1390
}
0 commit comments