Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
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: 2 additions & 2 deletions core/src/main/scala/org/apache/spark/TaskEndReason.scala
Original file line number Diff line number Diff line change
Expand Up @@ -117,8 +117,8 @@ case object TaskKilled extends TaskFailedReason {
* the task crashed the JVM.
*/
@DeveloperApi
case object ExecutorLostFailure extends TaskFailedReason {
override def toErrorString: String = "ExecutorLostFailure (executor lost)"
case class ExecutorLostFailure(execId: String) extends TaskFailedReason {
override def toErrorString: String = s"ExecutorLostFailure (executor ${execId} lost)"
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -706,7 +706,7 @@ private[spark] class TaskSetManager(
}
// Also re-enqueue any tasks that were running on the node
for ((tid, info) <- taskInfos if info.running && info.executorId == execId) {
handleFailedTask(tid, TaskState.FAILED, ExecutorLostFailure)
handleFailedTask(tid, TaskState.FAILED, ExecutorLostFailure(execId))
}
// recalculate valid locality levels and waits when executor is lost
recomputeLocality()
Expand Down
8 changes: 6 additions & 2 deletions core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
Original file line number Diff line number Diff line change
Expand Up @@ -272,7 +272,7 @@ private[spark] object JsonProtocol {

def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = {
val reason = Utils.getFormattedClassName(taskEndReason)
val json = taskEndReason match {
val json: JObject = taskEndReason match {
case fetchFailed: FetchFailed =>
val blockManagerAddress = Option(fetchFailed.bmAddress).
map(blockManagerIdToJson).getOrElse(JNothing)
Expand All @@ -287,6 +287,8 @@ private[spark] object JsonProtocol {
("Description" -> exceptionFailure.description) ~
("Stack Trace" -> stackTrace) ~
("Metrics" -> metrics)
case ExecutorLostFailure(executorId) =>
("Executor ID" -> executorId)
case _ => Utils.emptyJson
}
("Reason" -> reason) ~ json
Expand Down Expand Up @@ -636,7 +638,9 @@ private[spark] object JsonProtocol {
new ExceptionFailure(className, description, stackTrace, metrics)
case `taskResultLost` => TaskResultLost
case `taskKilled` => TaskKilled
case `executorLostFailure` => ExecutorLostFailure
case `executorLostFailure` =>
val executorId = (json \ "Executor ID").extract[String]
Copy link
Contributor

Choose a reason for hiding this comment

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

Yes this needs to be backward compatible.

Copy link
Contributor

Choose a reason for hiding this comment

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

I guess this can just be extractOpt with a default executor ID of "unknown"

Copy link
Contributor

Choose a reason for hiding this comment

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

There's a Utils.jsonOption that we can use here

ExecutorLostFailure(executorId)
case `unknownReason` => UnknownReason
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc
new ExceptionFailure("Exception", "description", null, None),
TaskResultLost,
TaskKilled,
ExecutorLostFailure,
ExecutorLostFailure("0"),
UnknownReason)
var failCount = 0
for (reason <- taskFailedReasons) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ class JsonProtocolSuite extends FunSuite {
testTaskEndReason(exceptionFailure)
testTaskEndReason(TaskResultLost)
testTaskEndReason(TaskKilled)
testTaskEndReason(ExecutorLostFailure)
testTaskEndReason(ExecutorLostFailure("100"))
testTaskEndReason(UnknownReason)

// BlockId
Expand Down Expand Up @@ -403,7 +403,8 @@ class JsonProtocolSuite extends FunSuite {
assertOptionEquals(r1.metrics, r2.metrics, assertTaskMetricsEquals)
case (TaskResultLost, TaskResultLost) =>
case (TaskKilled, TaskKilled) =>
case (ExecutorLostFailure, ExecutorLostFailure) =>
case (ExecutorLostFailure(execId1), ExecutorLostFailure(execId2)) =>
assert(execId1 === execId2)
case (UnknownReason, UnknownReason) =>
case _ => fail("Task end reasons don't match in types!")
}
Expand Down