Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-15865][CORE] Blacklist should not result in job hanging with less than 4 executors #13603

Closed
wants to merge 36 commits into from
Closed
Show file tree
Hide file tree
Changes from 32 commits
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
3f46275
if all executors have been blacklisted for a ask, abort stage (instea…
squito Jun 10, 2016
f870bde
Merge branch 'master' into progress_w_few_execs_and_blacklist
squito Jun 20, 2016
6b80b28
another test case
squito Jun 20, 2016
64ab7fb
move check for unschedulability into TaskSetManager
squito Jun 20, 2016
3edb6fe
overloooked cleanup
squito Jun 20, 2016
6362b28
rename taskId -> partitionId for failedExecutors
squito Jun 21, 2016
3da14ee
review feedback
squito Jun 21, 2016
060dbfe
fix partitionId in FakeTask
squito Jun 21, 2016
fd48403
need to stop taskscheduler and dagscheduler
squito Jun 21, 2016
c1aabea
update tests per suggestions
squito Jun 21, 2016
9efd95d
update comment
squito Jun 21, 2016
158775c
asserts for not-serializable tasks
squito Jun 21, 2016
e880495
fix assertion
squito Jun 21, 2016
39d18ed
refactor and fix tests
squito Jun 21, 2016
195408b
make sure executorsByHost and localityLevels updated on new executors…
squito Jun 21, 2016
40b8ee7
slightly expand test
squito Jun 21, 2016
7ca39a0
Revert "rename taskId -> partitionId for failedExecutors"
squito Jun 22, 2016
39fbf72
fix the taskId / partitionId mixup
squito Jun 22, 2016
87d6185
Merge branch 'master' into progress_w_few_execs_and_blacklist
squito Jun 22, 2016
c606dc3
Merge branch 'master' into progress_w_few_execs_and_blacklist
squito Jun 22, 2016
2f83927
dont start revive thread in constructor, needs to wait until executor…
squito Jun 22, 2016
635edec
add test for blacklist + locality + schedulability
squito Jun 23, 2016
2d52962
Merge branch 'SPARK-16106_executorByHosts' into progress_w_few_execs_…
squito Jun 23, 2016
9684eb7
update comment
squito Jun 27, 2016
28a00db
Merge branch 'master' into progress_w_few_execs_and_blacklist
squito Jun 27, 2016
e8e4fe5
fix merge
squito Jun 27, 2016
a5f7eb4
Merge branch 'master' into progress_w_few_execs_and_blacklist
squito Jun 28, 2016
b5b0f3f
fix merge
squito Jun 28, 2016
6dd93af
update comment
squito Jun 28, 2016
f4e95c6
remove clock stuff, not necessary in these tests
squito Jun 28, 2016
860ee1a
unused import
squito Jun 28, 2016
60cd959
simplify now that SPARK-16106 is in
squito Jun 28, 2016
9665029
review feedback
squito Jun 28, 2016
96049cd
review feedback
squito Jun 29, 2016
ed71c99
change pendingTask to a val
squito Jun 29, 2016
ed413ce
simplify
squito Jun 29, 2016
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 4 additions & 0 deletions core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,10 @@ import org.apache.spark.annotation.DeveloperApi
@DeveloperApi
class TaskInfo(
val taskId: Long,
/**
* The index of this task within its task set. Not necessarily the same as the ID of the RDD
* partition that the task is computing.
*/
val index: Int,
val attemptNumber: Int,
val launchTime: Long,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.scheduler.TaskLocality.TaskLocality
import org.apache.spark.scheduler.local.LocalSchedulerBackend
import org.apache.spark.storage.BlockManagerId
import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils}
import org.apache.spark.util._
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

undo this change? (nice to have the explicit imports as long as they're short)


/**
* Schedules tasks for multiple types of clusters by acting through a SchedulerBackend.
Expand Down Expand Up @@ -279,6 +279,9 @@ private[spark] class TaskSchedulerImpl(
}
}
}
if (!launchedTask) {
taskSet.abortIfTaskSetCompletelyBlacklisted(executorsByHost)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Based on our discussion yesterday, should this use executorIdToHost instead of executorsByHost? (that would also be marginally easier to parse in abortIfTaskSetCompletelyBlacklisted)

}
return launchedTask
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ private[spark] class TaskSetManager(
val copiesRunning = new Array[Int](numTasks)
val successful = new Array[Boolean](numTasks)
private val numFailures = new Array[Int](numTasks)
// key is taskId, value is a Map of executor id to when it failed
// key is taskId (aka TaskInfo.index), value is a Map of executor id to when it failed
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

here (and executorIsBlacklisted) is what got me confused about the meaning of these identifiers. I knew it was defintiely not the globally unique taskId, aka TID.

private val failedExecutors = new HashMap[Int, HashMap[String, Long]]()

val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil)
Expand Down Expand Up @@ -270,7 +270,7 @@ private[spark] class TaskSetManager(
* Is this re-execution of a failed task on an executor it already failed in before
* EXECUTOR_TASK_BLACKLIST_TIMEOUT has elapsed ?
*/
private def executorIsBlacklisted(execId: String, taskId: Int): Boolean = {
private[scheduler] def executorIsBlacklisted(execId: String, taskId: Int): Boolean = {
if (failedExecutors.contains(taskId)) {
val failed = failedExecutors.get(taskId).get

Expand Down Expand Up @@ -575,6 +575,61 @@ private[spark] class TaskSetManager(
index
}

/**
* Check whether the given task set has been blacklisted to the point that it can't run anywhere.
*
* It is possible that this taskset has become impossible to schedule *anywhere* due to the
* blacklist. The most common scenario would be if there are fewer executors than
* spark.task.maxFailures. We need to detect this so we can fail the task set, otherwise the job
* will hang.
*
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The next par. is a little hard to understand. How about something like (also is the below correct? It's a little different from what you had):

There's a tradeoff here: we could make sure all tasks in the task set are schedulable, but that would add extra time to each iteration of the scheduling loop. Here, we take the approach of making sure at least one of the unscheduled tasks is schedulable. This means we may not detect the hang as quickly as we could have, but we'll always detect the hang eventually, and the method is faster in the typical case. In the worst case, this method can take O(maxTaskFailures) time, but it will be faster when there haven't been any task failures (this is because the method picks on unscheduled task, and then iterates through each executor until it finds one that the task hasn't failed on already).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, much better. Also good point that its only O(maxTaskFailures), I hadn't realized that.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After further thought I think this is actually O(maxTaskFailures + # tasks) right? Since you might need to first iterate through all of the tasks to find one that hasn't started yet, and then iterate through some failed executors?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ah right, I forgot about the cost of pollPendingTask

* There's a tradeoff here: we could make sure all tasks in the task set are schedulable, but that
* would add extra time to each iteration of the scheduling loop. Here, we take the approach of
* making sure at least one of the unscheduled tasks is schedulable. This means we may not detect
* the hang as quickly as we could have, but we'll always detect the hang eventually, and the
* method is faster in the typical case. In the worst case, this method can take
* O(maxTaskFailures + numTasks) time, but it will be faster when there haven't been any task
* failures (this is because the method picks on unscheduled task, and then iterates through each
* executor until it finds one that the task hasn't failed on already).
*/
private[scheduler] def abortIfTaskSetCompletelyBlacklisted(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you could shorten this name to eliminate "TaskSet" (which I think is leftover from when it was in TaskSchedulerImpl)

executorsByHost: HashMap[String, HashSet[String]]): Unit = {

def pendingTask: Option[Int] = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

gah sorry one more tiny thing: can this just be a val?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you can't just change this to a val with nothing else, b/c of the return when we find the task. Though you could make it a val by changing the inner-logic, with a var keepGoing in the while loop or something. I actually changed it once and couldn't really make up my mind which version was cleaner ... in the end I felt an inner function wasn't so bad, but happy to change it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I pushed a commit with changing it to a val, so you can see both options. easy enough to back out that last commit.

Copy link
Contributor

@kayousterhout kayousterhout Jun 29, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what about

val pendingTask: Option[Int] = allPendingTasks.lastIndexWhere { indexInTaskSet =>
copiesRunning(indexInTaskSet) == 0 && !successful(indexInTaskSet)
}.map(allPendingTasks(_))

(I realize we're really in the weeds here so whatever you prefer here is fine)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh I didn't even know about lastIndexWhere! thanks, simpler, and despite being a minor point I appreciate learning something new :)

sorry I think I just got on the wrong track in this while thinking about doing the lazy-removal here as well, and when I decided against it never stepped back to simplify it.

// usually this will just take the last pending task, but because of the lazy removal
// from each list, we may need to go deeper in the list. We poll from the end because
// failed tasks are put back at the end of allPendingTasks, so we're more likely to find
// an unschedulable task this way.
var indexOffset = allPendingTasks.size
while (indexOffset > 0) {
indexOffset -= 1
val indexInTaskSet = allPendingTasks(indexOffset)
if (copiesRunning(indexInTaskSet) == 0 && !successful(indexInTaskSet)) {
return Some(indexInTaskSet)
}
}
None
}

// If no executors have registered yet, don't abort the stage, just wait. We probably
// got here because a task set was added before the executors registered.
if (executorsByHost.nonEmpty) {
// take any task that needs to be scheduled, and see if we can find some executor it *could*
// run on
pendingTask.foreach { taskId =>
executorsByHost.foreach { case (host, execs) =>
execs.foreach { exec =>
if (!executorIsBlacklisted(exec, taskId)) {
return
}
}
}
abort(s"Aborting ${taskSet} because Task $taskId (partition " +
s"${tasks(taskId).partitionId}) cannot be scheduled on any executor due to blacklists.")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe include the executors here (esp. since this is something users might see)?

s"Aborting ${taskSet} because task $taskId (partition ${tasks(taskId).partitionId}) has already failed on executors ${executorsByHost.values.flatten.mkString(",")}, and no other executors are available."

}
}
}

/**
* Marks the task as getting result and notifies the DAG Scheduler
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ class ExecutorSuite extends SparkFunSuite {
when(mockEnv.closureSerializer).thenReturn(serializer)
val serializedTask =
Task.serializeWithDependencies(
new FakeTask(0),
new FakeTask(0, 0),
HashMap[String, Long](),
HashMap[String, Long](),
serializer.newInstance())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,31 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM
assertDataStructuresEmpty(noFailure = true)
}

// Make sure that if we've failed on all executors, but haven't hit task.maxFailures yet, the job
// doesn't hang
testScheduler(
"SPARK-15865 Progress with fewer executors than maxTaskFailures",
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm pretty sure the previous flakiness in this test suite is unrelated to the test I'm adding here, but we should probably wait to merge this until after #13565 just to be safe. (at least, its unrelated to the delay-scheduling-revive-offers issue, which was the major previous flakiness -- potentially still suffers from the DAGScheduler / assertDataStructuresEmpty race, but that is quite rare.) I did run this test 5K times on my laptop and it passed (though that hasn't always been enough to discover some of the races).

extraConfs = Seq(
// set this to something much longer than the test duration
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add "so that executors don't get removed from the blacklist during the test"

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like you added this in the other test -- I know it's redundant here but still helpful for reference I think

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

doh, I thought I had changed it everywhere earlier, sorry, done now

"spark.scheduler.executorTaskBlacklistTime" -> "10000000",
"spark.testing.nHosts" -> "2",
"spark.testing.nExecutorsPerHost" -> "1",
"spark.testing.nCoresPerExecutor" -> "1"
)
) {
def runBackend(): Unit = {
val (taskDescription, _) = backend.beginTask()
backend.taskFailed(taskDescription, new RuntimeException("test task failure"))
}
withBackend(runBackend _) {
val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray)
Await.ready(jobFuture, duration)
val pattern = ("Aborting TaskSet 0.0 because Task .* " +
"cannot be scheduled on any executor due to blacklists").r
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(just noting you'll need to update this if you change the message above)

assert(pattern.findFirstIn(failure.getMessage).isDefined)
}
assertDataStructuresEmpty(noFailure = false)
}
}

class MultiExecutorMockBackend(
Expand Down
5 changes: 3 additions & 2 deletions core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,8 @@ import org.apache.spark.TaskContext

class FakeTask(
stageId: Int,
prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0, 0) {
partitionId: Int,
prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0, partitionId) {
override def runTask(context: TaskContext): Int = 0
override def preferredLocations: Seq[TaskLocation] = prefLocs
}
Expand All @@ -40,7 +41,7 @@ object FakeTask {
throw new IllegalArgumentException("Wrong number of task locations")
}
val tasks = Array.tabulate[Task[_]](numTasks) { i =>
new FakeTask(i, if (prefLocs.size != 0) prefLocs(i) else Nil)
new FakeTask(0, i, if (prefLocs.size != 0) prefLocs(i) else Nil)
}
new TaskSet(tasks, 0, stageAttemptId, 0, null)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext {
def createTaskSetManager(stageId: Int, numTasks: Int, taskScheduler: TaskSchedulerImpl)
: TaskSetManager = {
val tasks = Array.tabulate[Task[_]](numTasks) { i =>
new FakeTask(i, Nil)
new FakeTask(stageId, i, Nil)
}
new TaskSetManager(taskScheduler, new TaskSet(tasks, stageId, 0, 0, null), 0)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -279,12 +279,6 @@ private[spark] abstract class MockBackend(
ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread")
private val reviveIntervalMs = conf.getTimeAsMs("spark.scheduler.revive.interval", "10ms")

reviveThread.scheduleAtFixedRate(new Runnable {
override def run(): Unit = Utils.tryLogNonFatalError {
reviveOffers()
}
}, 0, reviveIntervalMs, TimeUnit.MILLISECONDS)

/**
* Test backends should call this to get a task that has been assigned to them by the scheduler.
* Each task should be responded to with either [[taskSuccess]] or [[taskFailed]].
Expand Down Expand Up @@ -348,7 +342,13 @@ private[spark] abstract class MockBackend(
assignedTasksWaitingToRun.nonEmpty
}

override def start(): Unit = {}
override def start(): Unit = {
reviveThread.scheduleAtFixedRate(new Runnable {
override def run(): Unit = Utils.tryLogNonFatalError {
reviveOffers()
}
}, 0, reviveIntervalMs, TimeUnit.MILLISECONDS)
}

override def stop(): Unit = {
reviveThread.shutdown()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -281,6 +281,96 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
assert(!failedTaskSet)
}

test("abort stage if executor loss results in unschedulability from previously failed tasks") {
// Make sure we can detect when a taskset becomes unschedulability from a blacklisting. This
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

unschedulability -> unschedulable

// test explores a particular corner case -- you may have one task fail, but still be
// schedulable on another executor. However, that executor may fail later on, leaving the
// first task with no place to run.
val taskScheduler = setupScheduler(
// set this to something much longer than the test duration
"spark.scheduler.executorTaskBlacklistTime" -> "10000000"
)

val taskSet = FakeTask.createTaskSet(2)
taskScheduler.submitTasks(taskSet)
val tsm = taskScheduler.taskSetManagerForAttempt(taskSet.stageId, taskSet.stageAttemptId).get

val firstTaskAttempts = taskScheduler.resourceOffers(Seq(
new WorkerOffer("executor0", "host0", 1),
new WorkerOffer("executor1", "host1", 1)
)).flatten
assert(Set("executor0", "executor1") === firstTaskAttempts.map(_.executorId).toSet)

// fail one of the tasks, but leave the other running
val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
taskScheduler.handleFailedTask(tsm, failedTask.taskId, TaskState.FAILED, TaskResultLost)
// at this point, our failed task could run on the other executor, so don't give up the task
// set yet.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does it make sense to verify !failedTaskSet here?

assert(!failedTaskSet)

// Now we fail our second executor. The other task can still run on executor1, so make an offer
// on that executor, and make sure that the other task (not the failed one) is assigned there
taskScheduler.executorLost("executor1", SlaveLost("oops"))
val nextTaskAttempts =
taskScheduler.resourceOffers(Seq(new WorkerOffer("executor0", "host0", 1))).flatten
// Note: Its OK if some future change makes this already realize the taskset has become
// unschedulable at this point (though in the current implementation, we're sure it will not)
assert(nextTaskAttempts.size === 1)
assert(nextTaskAttempts.head.executorId === "executor0")
assert(nextTaskAttempts.head.attemptNumber === 1)
assert(nextTaskAttempts.head.index != failedTask.index)

// now we should definitely realize that our task set is unschedulable, because the only
// task left can't be scheduled on any executors due to the blacklist
taskScheduler.resourceOffers(Seq(new WorkerOffer("executor0", "host0", 1)))
sc.listenerBus.waitUntilEmpty(100000)
assert(tsm.isZombie)
assert(failedTaskSet)
val idx = failedTask.index
assert(failedTaskSetReason.contains(s"Aborting TaskSet 0.0 because Task $idx (partition $idx)" +
s" cannot be scheduled on any executor due to blacklists."))
}

test("don't abort if there is an executor available, though it hasn't had scheduled tasks yet") {
// interaction of SPARK-15865 & SPARK-16106
// if we have a small number of tasks, we might be able to schedule them all on the first
// executor. But if those tasks fail, we should still realize there is another executor
// available and not bail on the job

val taskScheduler = setupScheduler(
// set this to something much longer than the test duration
"spark.scheduler.executorTaskBlacklistTime" -> "10000000"
)

val taskSet = FakeTask.createTaskSet(2, (0 until 2).map { _ => Seq(TaskLocation("host0")) }: _*)
taskScheduler.submitTasks(taskSet)
val tsm = taskScheduler.taskSetManagerForAttempt(taskSet.stageId, taskSet.stageAttemptId).get

val offers = Seq(
// each offer has more than enough free cores for the entire task set, so we schedule
// all tasks on one executor
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We schedule them all on host0 b/c of the locality preference right? I thought we try to spread the tasks out across executors (even if they'd all fit on one) otherwise?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, because of locality preference. I'll update the comment to make that clear

new WorkerOffer("executor0", "host0", 4),
new WorkerOffer("executor1", "host1", 4)
)
val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten
assert(firstTaskAttempts.size == 2)
firstTaskAttempts.foreach { taskAttempt => assert("executor0" === taskAttempt.executorId) }

// fail all the tasks on the bad executor
firstTaskAttempts.foreach { taskAttempt =>
taskScheduler.handleFailedTask(tsm, taskAttempt.taskId, TaskState.FAILED, TaskResultLost)
}

// Here is the main check of this test -- we have the same offers again, and we schedule it
// successfully. Because the scheduler first tries to schedule with locality in mind, at first
// it won't schedule anything on executor1. But despite that, we don't abort the job. Then the
// scheduler tries for ANY locality, and successfully schedules tasks on executor1.
val secondTaskAttempts = taskScheduler.resourceOffers(offers).flatten
assert(secondTaskAttempts.size == 2)
secondTaskAttempts.foreach { taskAttempt => assert("executor1" === taskAttempt.executorId) }
assert(!failedTaskSet)
}

test("SPARK-16106 locality levels updated if executor added to existing host") {
val taskScheduler = setupScheduler()

Expand Down