You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "SuYan (JIRA)" <ji...@apache.org> on 2017/09/25 11:39:00 UTC

[jira] [Created] (SPARK-22116) Should ignore fetchFaileException if caused by kill event

SuYan created SPARK-22116:
-----------------------------

             Summary: Should ignore fetchFaileException if caused by kill event
                 Key: SPARK-22116
                 URL: https://issues.apache.org/jira/browse/SPARK-22116
             Project: Spark
          Issue Type: Bug
          Components: Spark Core
    Affects Versions: 2.1.1
            Reporter: SuYan


2017-09-22,10:24:56,878 INFO org.apache.spark.scheduler.TaskSetManager: Killing attempt 0 for task 169.0 in stage 2.0 (TID 429) on c3352.bj as the attempt 1 succeeded on c3943.bj
2017-09-22,10:24:56,878 INFO org.apache.spark.scheduler.TaskSetManager: Finished task 169.1 in stage 2.0 (TID 1183) in 764 ms on c3943.bj (executor 361) (881/900)

2017-09-22,10:24:56,911 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 169.0 in stage 2.0 (TID 429, c3352.bj, executor 378): FetchFailed(BlockManagerId(378, c3352.bj, 52739, None), shuffleId=0, mapId=153, reduceId=169, message=
org.apache.spark.shuffle.FetchFailedException: Error in opening FileSegmentManagedBuffer{file=/home/work/hdd4/yarn/c3hadoop/nodemanager/usercache/h_user_profile/appcache/application_1505730831071_156911/blockmgr-4d51401d-315a-4d84-92d4-7d525691f9d5/24/shuffle_0_153_0.data, offset=2179859, length=12866}
at org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:438)
at org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:378)
at org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:59)
at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
at org.apache.spark.shuffle.BlockStoreShuffleReader$ProcessFetchFailedIterator$$anonfun$hasNext$1.apply$mcZ$sp(BlockStoreShuffleReader.scala:

{code}
        case t: Throwable if hasFetchFailure && !Utils.isFatalError(t) =>
          val reason = task.context.fetchFailed.get.toTaskFailedReason
          if (!t.isInstanceOf[FetchFailedException]) {
            // there was a fetch failure in the task, but some user code wrapped that exception
            // and threw something else.  Regardless, we treat it as a fetch failure.
            val fetchFailedCls = classOf[FetchFailedException].getName
            logWarning(s"TID ${taskId} encountered a ${fetchFailedCls} and " +
              s"failed, but the ${fetchFailedCls} was hidden by another " +
              s"exception.  Spark is handling this like a fetch failure and ignoring the " +
              s"other exception: $t")
          }
          setTaskFinishedAndClearInterruptStatus()
          execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))

        case t: TaskKilledException =>
          logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}")
          setTaskFinishedAndClearInterruptStatus()
          execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled(t.reason)))

        case _: InterruptedException | NonFatal(_) if
            task != null && task.reasonIfKilled.isDefined =>
{code}

Solution:
 may put killed as the first level to handle?  and also check ignore failed task if this index task have already success? 





--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org