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

    https://github.com/apache/spark/pull/214#discussion_r10949633
  
    --- Diff: 
core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -219,18 +229,40 @@ private[spark] class TaskSchedulerImpl(
             taskSet.parent.name, taskSet.name, taskSet.runningTasks))
         }
     
    +    val ser = SparkEnv.get.closureSerializer.newInstance()
         // Take each TaskSet in our scheduling order, and then offer it each 
node in increasing order
         // of locality levels so that it gets a chance to launch local tasks 
on all of them.
         var launchedTask = false
    +    val serializingTaskNum = new AtomicLong(0)
         for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) {
           do {
             launchedTask = false
             for (i <- 0 until shuffledOffers.size) {
               val execId = shuffledOffers(i).executorId
               val host = shuffledOffers(i).host
    -          for (task <- taskSet.resourceOffer(execId, host, 
availableCpus(i), maxLocality)) {
    -            tasks(i) += task
    -            val tid = task.taskId
    +          for (taskDesc <- taskSet.resourceOffer(execId, host, 
availableCpus(i), maxLocality)) {
    +            serializingTaskNum.getAndIncrement()
    +            serializeWorkerPool.execute(new Runnable {
    +              override def run() {
    +                // Serialize and return the task
    +                val startTime = System.currentTimeMillis()
    +                // We rely on the DAGScheduler to catch non-serializable 
closures and RDDs, so in here
    +                // we assume the task can be serialized without exceptions.
    +                val serializedTask = Task.serializeWithDependencies(
    +                  taskDesc.taskObject, sc.addedFiles, sc.addedJars, ser)
    +                val timeTaken = System.currentTimeMillis() - startTime
    +                logInfo("Serialized task %s as %d bytes in %d ms".format(
    +                  taskDesc.taskName, serializedTask.limit, timeTaken))
    +                val task = new TaskDescription(taskDesc.taskId, 
taskDesc.executorId,
    +                    taskDesc.taskName, taskDesc.index, serializedTask)
    +                tasks.synchronized {
    --- End diff --
    
    hmm, since it is running in a different thread, no need to set spark env, 
etc ?


---
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