Details
-
Bug
-
Status: Resolved
-
Critical
-
Resolution: Fixed
-
1.0.2, 1.1.1, 1.2.0, 1.3.0
-
None
Description
When speculative execution is enabled (spark.speculation=true), jobs that save output files may report that they have completed successfully even though some output partitions written by speculative tasks may be missing.
Reproduction
This symptom was reported to me by a Spark user and I've been doing my own investigation to try to come up with an in-house reproduction.
I'm still working on a reliable local reproduction for this issue, which is a little tricky because Spark won't schedule speculated tasks on the same host as the original task, so you need an actual (or containerized) multi-host cluster to test speculation. Here's a simple reproduction of some of the symptoms on EC2, which can be run in spark-shell with --conf spark.speculation=true:
// Rig a job such that all but one of the tasks complete instantly // and one task runs for 20 seconds on its first attempt and instantly // on its second attempt: val numTasks = 100 sc.parallelize(1 to numTasks, numTasks).repartition(2).mapPartitionsWithContext { case (ctx, iter) => if (ctx.partitionId == 0) { // If this is the one task that should run really slow if (ctx.attemptId == 0) { // If this is the first attempt, run slow Thread.sleep(20 * 1000) } } iter }.map(x => (x, x)).saveAsTextFile("/test4")
When I run this, I end up with a job that completes quickly (due to speculation) but reports failures from the speculated task:
[...] 14/12/11 01:41:13 INFO scheduler.TaskSetManager: Finished task 37.1 in stage 3.0 (TID 411) in 131 ms on ip-172-31-8-164.us-west-2.compute.internal (100/100) 14/12/11 01:41:13 INFO scheduler.DAGScheduler: Stage 3 (saveAsTextFile at <console>:22) finished in 0.856 s 14/12/11 01:41:13 INFO spark.SparkContext: Job finished: saveAsTextFile at <console>:22, took 0.885438374 s 14/12/11 01:41:13 INFO scheduler.TaskSetManager: Ignoring task-finished event for 70.1 in stage 3.0 because task 70 has already completed successfully scala> 14/12/11 01:41:13 WARN scheduler.TaskSetManager: Lost task 49.1 in stage 3.0 (TID 413, ip-172-31-8-164.us-west-2.compute.internal): java.io.IOException: Failed to save output of task: attempt_201412110141_0003_m_000049_413 org.apache.hadoop.mapred.FileOutputCommitter.moveTaskOutputs(FileOutputCommitter.java:160) org.apache.hadoop.mapred.FileOutputCommitter.moveTaskOutputs(FileOutputCommitter.java:172) org.apache.hadoop.mapred.FileOutputCommitter.commitTask(FileOutputCommitter.java:132) org.apache.spark.SparkHadoopWriter.commit(SparkHadoopWriter.scala:109) org.apache.spark.rdd.PairRDDFunctions$$anonfun$13.apply(PairRDDFunctions.scala:991) org.apache.spark.rdd.PairRDDFunctions$$anonfun$13.apply(PairRDDFunctions.scala:974) org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:62) org.apache.spark.scheduler.Task.run(Task.scala:54) org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:177) java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) java.lang.Thread.run(Thread.java:745)
One interesting thing to note about this stack trace: if we look at FileOutputCommitter.java:160 (link), this point in the execution seems to correspond to a case where a task completes, attempts to commit its output, fails for some reason, then deletes the destination file, tries again, and fails:
if (fs.isFile(taskOutput)) { 152 Path finalOutputPath = getFinalPath(jobOutputDir, taskOutput, 153 getTempTaskOutputPath(context)); 154 if (!fs.rename(taskOutput, finalOutputPath)) { 155 if (!fs.delete(finalOutputPath, true)) { 156 throw new IOException("Failed to delete earlier output of task: " + 157 attemptId); 158 } 159 if (!fs.rename(taskOutput, finalOutputPath)) { 160 throw new IOException("Failed to save output of task: " + 161 attemptId); 162 } 163 }
This could explain why the output file is missing: the second copy of the task keeps running after the job completes and deletes the output written by the other task after failing to commit its own copy of the output.
There are still a few open questions about how exactly we get into this scenario:
Why is the second copy of the task allowed to commit its output after the other task / the job has successfully completed?
To check whether a task's temporary output should be committed, SparkHadoopWriter calls FileOutputCommitter.needsTaskCommit(), which returns true if the tasks's temporary output exists (link). Tihs does not seem to check whether the destination already exists. This means that needsTaskCommit can return true for speculative tasks.
Why does the rename fail?
I think that what's happening is that the temporary task output files are being deleted once the job has completed, which is causing the rename to fail because FileOutputCommitter.commitTask doesn't seem to guard against missing output files.
I'm not sure about this, though, since the stack trace seems to imply that the temporary output file existed. Maybe the filesystem methods are returning stale metadata? Maybe there's a race? I think a race condition seems pretty unlikely, since the time-scale at which it would have to happen doesn't sync up with the scale of the timestamps that I saw in the user report.
Possible Fixes:
The root problem here might be that speculative copies of tasks are somehow allowed to commit their output. We might be able to fix this by centralizing the "should this task commit its output" decision at the driver.
(I have more concrete suggestions of how to do this; to be posted soon)