[
https://issues.apache.org/jira/browse/SPARK-2984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14134073#comment-14134073
]
Gregory Phillips edited comment on SPARK-2984 at 9/15/14 4:43 PM:
--
I'm running into this as well. But to respond to this theory:
{quote}
I think this may be related to spark.speculation. I think the error condition
might manifest in this circumstance:
1) task T starts on a executor E1
2) it takes a long time, so task T' is started on another executor E2
3) T finishes in E1 so moves its data from _temporary to the final destination
and deletes the _temporary directory during cleanup
4) T' finishes in E2 and attempts to move its data from _temporary, but those
files no longer exist! exception
{quote}
Speculation is not necessary for this to occur. I am consistently running into
this while testing some code against local without speculation where I am
trying to download, manipulate and merge 2 sets of data from S3 and serialize
the resulting RDD using saveAsTextFile back to S3:
{code}
Job aborted due to stage failure: Task 3.0:754 failed 1 times, most recent
failure: Exception failure in TID 762 on host localhost:
java.io.FileNotFoundException:
s3n://bucket/_temporary/_attempt_201409151537__m_000754_762/part-00754.deflate:
No such file or directory.
org.apache.hadoop.fs.s3native.NativeS3FileSystem.getFileStatus(NativeS3FileSystem.java:340)
org.apache.hadoop.mapred.FileOutputCommitter.moveTaskOutputs(FileOutputCommitter.java:165)
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:786)
org.apache.spark.rdd.PairRDDFunctions$$anonfun$13.apply(PairRDDFunctions.scala:769)
org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:111)
org.apache.spark.scheduler.Task.run(Task.scala:51)
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:183)
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
java.lang.Thread.run(Thread.java:744) Driver stacktrace:
{code}
I'm happy to provide more information or help investigate further to figure
this one out.
Edit: I forgot to mention that the file in question actually did exist on S3
when I checked after receiving this exception.
was (Author: gphil):
I'm running into this as well. But to respond to this theory:
{quote}
I think this may be related to spark.speculation. I think the error condition
might manifest in this circumstance:
1) task T starts on a executor E1
2) it takes a long time, so task T' is started on another executor E2
3) T finishes in E1 so moves its data from _temporary to the final destination
and deletes the _temporary directory during cleanup
4) T' finishes in E2 and attempts to move its data from _temporary, but those
files no longer exist! exception
{quote}
Speculation is not necessary for this to occur. I am consistently running into
this while testing some code against local without speculation where I am
trying to download, manipulate and merge 2 sets of data from S3 and serialize
the resulting RDD using saveAsTextFile back to S3:
{code}
Job aborted due to stage failure: Task 3.0:754 failed 1 times, most recent
failure: Exception failure in TID 762 on host localhost:
java.io.FileNotFoundException:
s3n://bucket/_temporary/_attempt_201409151537__m_000754_762/part-00754.deflate:
No such file or directory.
org.apache.hadoop.fs.s3native.NativeS3FileSystem.getFileStatus(NativeS3FileSystem.java:340)
org.apache.hadoop.mapred.FileOutputCommitter.moveTaskOutputs(FileOutputCommitter.java:165)
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:786)
org.apache.spark.rdd.PairRDDFunctions$$anonfun$13.apply(PairRDDFunctions.scala:769)
org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:111)
org.apache.spark.scheduler.Task.run(Task.scala:51)
org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:183)
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
java.lang.Thread.run(Thread.java:744) Driver stacktrace:
{code}
I'm happy to provide more information or help investigate further to figure
this one out.
FileNotFoundException on _temporary directory