aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
blob: e52cb998bdf42df09e45b23b2692c9aa0b497a13 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.scheduler.local

import java.nio.ByteBuffer
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap

import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState}
import org.apache.spark.TaskState.TaskState
import org.apache.spark.scheduler.{Task, TaskResult, TaskSet}
import org.apache.spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager}


private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet)
  extends TaskSetManager with Logging {

  var parent: Schedulable = null
  var weight: Int = 1
  var minShare: Int = 0
  var runningTasks: Int = 0
  var priority: Int = taskSet.priority
  var stageId: Int = taskSet.stageId
  var name: String = "TaskSet_" + taskSet.stageId.toString

  var failCount = new Array[Int](taskSet.tasks.size)
  val taskInfos = new HashMap[Long, TaskInfo]
  val numTasks = taskSet.tasks.size
  var numFinished = 0
  val env = SparkEnv.get
  val ser = env.closureSerializer.newInstance()
  val copiesRunning = new Array[Int](numTasks)
  val finished = new Array[Boolean](numTasks)
  val numFailures = new Array[Int](numTasks)
  val MAX_TASK_FAILURES = sched.maxFailures

  override def increaseRunningTasks(taskNum: Int): Unit = {
    runningTasks += taskNum
    if (parent != null) {
     parent.increaseRunningTasks(taskNum)
    }
  }

  override def decreaseRunningTasks(taskNum: Int): Unit = {
    runningTasks -= taskNum
    if (parent != null) {
      parent.decreaseRunningTasks(taskNum)
    }
  }

  override def addSchedulable(schedulable: Schedulable): Unit = {
    // nothing
  }

  override def removeSchedulable(schedulable: Schedulable): Unit = {
    // nothing
  }

  override def getSchedulableByName(name: String): Schedulable = {
    return null
  }

  override def executorLost(executorId: String, host: String): Unit = {
    // nothing
  }

  override def checkSpeculatableTasks() = true

  override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = {
    var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager]
    sortedTaskSetQueue += this
    return sortedTaskSetQueue
  }

  override def hasPendingTasks() = true

  def findTask(): Option[Int] = {
    for (i <- 0 to numTasks-1) {
      if (copiesRunning(i) == 0 && !finished(i)) {
        return Some(i)
      }
    }
    return None
  }

  override def resourceOffer(
      execId: String,
      host: String,
      availableCpus: Int,
      maxLocality: TaskLocality.TaskLocality)
    : Option[TaskDescription] =
  {
    SparkEnv.set(sched.env)
    logDebug("availableCpus:%d, numFinished:%d, numTasks:%d".format(
      availableCpus.toInt, numFinished, numTasks))
    if (availableCpus > 0 && numFinished < numTasks) {
      findTask() match {
        case Some(index) =>
          val taskId = sched.attemptId.getAndIncrement()
          val task = taskSet.tasks(index)
          val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1",
            TaskLocality.NODE_LOCAL)
          taskInfos(taskId) = info
          // 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 bytes = Task.serializeWithDependencies(
            task, sched.sc.addedFiles, sched.sc.addedJars, ser)
          logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes")
          val taskName = "task %s:%d".format(taskSet.id, index)
          copiesRunning(index) += 1
          increaseRunningTasks(1)
          taskStarted(task, info)
          return Some(new TaskDescription(taskId, null, taskName, index, bytes))
        case None => {}
      }
    }
    return None
  }

  override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
    SparkEnv.set(env)
    state match {
      case TaskState.FINISHED =>
        taskEnded(tid, state, serializedData)
      case TaskState.FAILED =>
        taskFailed(tid, state, serializedData)
      case _ => {}
    }
  }

  def taskStarted(task: Task[_], info: TaskInfo) {
    sched.listener.taskStarted(task, info)
  }

  def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) {
    val info = taskInfos(tid)
    val index = info.index
    val task = taskSet.tasks(index)
    info.markSuccessful()
    val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader)
    result.metrics.resultSize = serializedData.limit()
    sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics)
    numFinished += 1
    decreaseRunningTasks(1)
    finished(index) = true
    if (numFinished == numTasks) {
      sched.taskSetFinished(this)
    }
  }

  def taskFailed(tid: Long, state: TaskState, serializedData: ByteBuffer) {
    val info = taskInfos(tid)
    val index = info.index
    val task = taskSet.tasks(index)
    info.markFailed()
    decreaseRunningTasks(1)
    val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](
      serializedData, getClass.getClassLoader)
    sched.listener.taskEnded(task, reason, null, null, info, reason.metrics.getOrElse(null))
    if (!finished(index)) {
      copiesRunning(index) -= 1
      numFailures(index) += 1
      val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString))
      logInfo("Loss was due to %s\n%s\n%s".format(
        reason.className, reason.description, locs.mkString("\n")))
      if (numFailures(index) > MAX_TASK_FAILURES) {
        val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(
          taskSet.id, index, 4, reason.description)
        decreaseRunningTasks(runningTasks)
        sched.listener.taskSetFailed(taskSet, errorMessage)
        // need to delete failed Taskset from schedule queue
        sched.taskSetFinished(this)
      }
    }
  }

  override def error(message: String) {
  }
}