Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/694#discussion_r12765502
  
    --- Diff: 
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
 ---
    @@ -140,8 +141,29 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, actorSystem: A
         // Launch tasks returned by a set of resource offers
         def launchTasks(tasks: Seq[Seq[TaskDescription]]) {
           for (task <- tasks.flatten) {
    -        freeCores(task.executorId) -= scheduler.CPUS_PER_TASK
    -        executorActor(task.executorId) ! LaunchTask(task)
    +        val ser = SparkEnv.get.closureSerializer.newInstance()
    +        val serializedTask = ser.serialize(task)
    +        if (serializedTask.limit >= akkaFrameSize - 1024) {
    +          val taskSetId = scheduler.taskIdToTaskSetId(task.taskId)
    +          scheduler.activeTaskSets.get(taskSetId).foreach { taskSet =>
    +            try {
    +              var msg = "Serialized task %s:%d was %d bytes which " +
    +                "exceeds spark.akka.frameSize (%d bytes). " +
    +                "Consider using broadcast variables for large values."
    +              msg = msg.format(task.taskId, task.index, 
serializedTask.limit, akkaFrameSize)
    +              taskSet.abort(msg)
    +            } catch {
    +              case e: Exception => logError("Exception in error callback", 
e)
    +            }
    +          }
    +          // scheduler.error(msg)
    +          // TODO: Need to throw an exception?
    +          // throw new SparkException(msg)
    --- End diff --
    
    Do we still need this code in here if we're aborting the TaskSet?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to