Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-14357

Tasks that fail due to CommitDeniedException (a side-effect of speculation) can cause job failure

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 1.5.2, 1.6.0, 1.6.1
    • 1.5.2, 1.6.2, 2.0.0
    • Spark Core
    • None

    Description

      Speculation can often result in a CommitDeniedException, but ideally this shouldn't result in the job failing. So changes were made along with SPARK-8167 to ensure that the CommitDeniedException is caught and given a failure reason that doesn't increment the failure count.

      However, I'm still noticing that this exception is causing jobs to fail using the 1.6.1 release version.

      16/04/04 11:36:02 ERROR InsertIntoHadoopFsRelation: Aborting job.
      org.apache.spark.SparkException: Job aborted due to stage failure: Task 18 in stage 315.0 failed 8 times, most recent failure: Lost task 18.8 in stage 315.0 (TID 100793, qaphdd099.quantium.com.au.local): org.apache.spark.SparkException: Task failed while writing rows.
              at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:272)
              at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150)
              at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150)
              at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
              at org.apache.spark.scheduler.Task.run(Task.scala:89)
              at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
              at java.lang.Thread.run(Thread.java:745)
      Caused by: java.lang.RuntimeException: Failed to commit task
              at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.commitTask$1(WriterContainer.scala:287)
              at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:267)
              ... 8 more
      Caused by: org.apache.spark.executor.CommitDeniedException: attempt_201604041136_0315_m_000018_8: Not committed because the driver did not authorize commit
              at org.apache.spark.mapred.SparkHadoopMapRedUtil$.commitTask(SparkHadoopMapRedUtil.scala:135)
              at org.apache.spark.sql.execution.datasources.BaseWriterContainer.commitTask(WriterContainer.scala:219)
              at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.commitTask$1(WriterContainer.scala:282)
              ... 9 more
      

      It seems to me that the CommitDeniedException gets wrapped into a RuntimeException at WriterContainer.scala#L286 and then into a SparkException at InsertIntoHadoopFsRelation.scala#L154 which results in it not being able to be handled properly at Executor.scala#L290

      The solution might be that this catch block should type match on the inner-most cause of an error?

      Attachments

        Issue Links

          Activity

            People

              jasonmoore2k Jason Moore
              jasonmoore2k Jason Moore
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: