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

    https://github.com/apache/spark/pull/15505#discussion_r99463382
  
    --- Diff: 
core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
 ---
    @@ -244,32 +245,45 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
         // Launch tasks returned by a set of resource offers
         private def launchTasks(tasks: Seq[Seq[TaskDescription]]) {
           for (task <- tasks.flatten) {
    -        val serializedTask = TaskDescription.encode(task)
    -        if (serializedTask.limit >= maxRpcMessageSize) {
    -          scheduler.taskIdToTaskSetManager.get(task.taskId).foreach { 
taskSetMgr =>
    -            try {
    -              var msg = "Serialized task %s:%d was %d bytes, which exceeds 
max allowed: " +
    -                "spark.rpc.message.maxSize (%d bytes). Consider increasing 
" +
    -                "spark.rpc.message.maxSize or using broadcast variables 
for large values."
    -              msg = msg.format(task.taskId, task.index, 
serializedTask.limit, maxRpcMessageSize)
    -              taskSetMgr.abort(msg)
    -            } catch {
    -              case e: Exception => logError("Exception in error callback", 
e)
    -            }
    -          }
    +        val serializedTask = try {
    +          TaskDescription.encode(task)
    +        } catch {
    +          case NonFatal(e) =>
    +            abortTaskSetManager(scheduler, task.taskId,
    --- End diff --
    
    If we do not deal with the  problem (2), it should only lead to performance 
degradation 
    when a serialization error occurred?
    or add the following code
    
    ```scala
    
        // Launch tasks returned by a set of resource offers
        private def launchTasks(tasks: Seq[Seq[TaskDescription]]) {
          val abortSet = new mutable.HashSet[TaskSetManager]()
          for (task <- tasks.flatten) {
            var serializedTask: ByteBuffer = null
            if (abortSet.isEmpty ||
              
!scheduler.taskIdToTaskSetManager.get(task.taskId).exists(_.isZombie)) {
              try {
                serializedTask = TaskDescription.encode(task)
              } catch {
                case NonFatal(e) =>
                  abortTaskSetManager(scheduler, task.taskId,
                    s"Failed to serialize task ${task.taskId}, not attempting 
to retry it.", Some(e))
                  scheduler.taskIdToTaskSetManager.get(task.taskId).foreach(t 
=> abortSet.add(t))
              }
            }
    
            if (serializedTask != null && serializedTask.limit >= 
maxRpcMessageSize) {
              val msg = "Serialized task %s:%d was %d bytes, which exceeds max 
allowed: " +
                "spark.rpc.message.maxSize (%d bytes). Consider increasing " +
                "spark.rpc.message.maxSize or using broadcast variables for 
large values."
              abortTaskSetManager(scheduler, task.taskId,
                msg.format(task.taskId, task.index, serializedTask.limit, 
maxRpcMessageSize))
              scheduler.taskIdToTaskSetManager.get(task.taskId).foreach(t => 
abortSet.add(t))
            } else if (serializedTask != null) {
              if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 
1024) {
                
scheduler.taskIdToTaskSetManager.get(task.taskId).filterNot(_.emittedTaskSizeWarning).
                  foreach { taskSetMgr =>
                    taskSetMgr.emittedTaskSizeWarning = true
                    val stageId = taskSetMgr.taskSet.stageId
                    logWarning(s"Stage $stageId contains a task of very large 
size " +
                      s"(${serializedTask.limit / 1024} KB). The maximum 
recommended task size is " +
                      s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.")
                  }
              }
              val executorData = executorDataMap(task.executorId)
              executorData.freeCores -= scheduler.CPUS_PER_TASK
    
              logDebug(s"Launching task ${task.taskId} on executor id: 
${task.executorId} " +
                s" hostname: ${executorData.executorHost}.")
    
              executorData.executorEndpoint.send(LaunchTask(new 
SerializableBuffer(serializedTask)))
            }
    
          }
        }
    ```



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to