@@ -27,10 +27,10 @@ import akka.actor._
27
27
import akka .pattern .ask
28
28
import akka .remote .{DisassociatedEvent , RemotingLifecycleEvent }
29
29
30
- import org .apache .spark .{Logging , SparkException , TaskState }
30
+ import org .apache .spark .{SparkEnv , Logging , SparkException , TaskState }
31
31
import org .apache .spark .scheduler .{SchedulerBackend , SlaveLost , TaskDescription , TaskSchedulerImpl , WorkerOffer }
32
32
import org .apache .spark .scheduler .cluster .CoarseGrainedClusterMessages ._
33
- import org .apache .spark .util .{AkkaUtils , Utils }
33
+ import org .apache .spark .util .{SerializableBuffer , AkkaUtils , Utils }
34
34
35
35
/**
36
36
* A scheduler backend that waits for coarse grained executors to connect to it through Akka.
@@ -48,6 +48,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
48
48
var totalCoreCount = new AtomicInteger (0 )
49
49
val conf = scheduler.sc.conf
50
50
private val timeout = AkkaUtils .askTimeout(conf)
51
+ private val akkaFrameSize = AkkaUtils .maxFrameSizeBytes(conf)
51
52
52
53
class DriverActor (sparkProperties : Seq [(String , String )]) extends Actor {
53
54
private val executorActor = new HashMap [String , ActorRef ]
@@ -141,7 +142,18 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
141
142
def launchTasks (tasks : Seq [Seq [TaskDescription ]]) {
142
143
for (task <- tasks.flatten) {
143
144
freeCores(task.executorId) -= scheduler.CPUS_PER_TASK
144
- executorActor(task.executorId) ! LaunchTask (task)
145
+ val ser = SparkEnv .get.closureSerializer.newInstance()
146
+ val taskBytes = ser.serialize(task).array()
147
+ val serializedTask = ser.serialize(taskBytes)
148
+ if (serializedTask.limit >= akkaFrameSize - 1024 ) {
149
+ var msg = " Serialized task %s:%d were %d bytes which " +
150
+ " exceeds spark.akka.frameSize (%d bytes)."
151
+ msg = msg.format(task.taskId, task.index, serializedTask.limit, akkaFrameSize)
152
+ val exception = new SparkException (msg)
153
+ logError(msg, exception)
154
+ throw exception
155
+ }
156
+ executorActor(task.executorId) ! LaunchTask (new SerializableBuffer (serializedTask))
145
157
}
146
158
}
147
159
0 commit comments