Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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
2 changes: 2 additions & 0 deletions core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -193,6 +193,8 @@ class BarrierTaskContext private[spark] (

override def isCompleted(): Boolean = taskContext.isCompleted()

override def isFailed(): Boolean = taskContext.isFailed()

override def isInterrupted(): Boolean = taskContext.isInterrupted()

override def addTaskCompletionListener(listener: TaskCompletionListener): this.type = {
Expand Down
5 changes: 5 additions & 0 deletions core/src/main/scala/org/apache/spark/TaskContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,11 @@ abstract class TaskContext extends Serializable {
*/
def isCompleted(): Boolean

/**
* Returns true if the task has failed.
*/
def isFailed(): Boolean

/**
* Returns true if the task has been killed.
*/
Expand Down
2 changes: 2 additions & 0 deletions core/src/main/scala/org/apache/spark/TaskContextImpl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -275,6 +275,8 @@ private[spark] class TaskContextImpl(
@GuardedBy("this")
override def isCompleted(): Boolean = synchronized(completed)

override def isFailed(): Boolean = synchronized(failureCauseOpt.isDefined)

override def isInterrupted(): Boolean = reasonIfKilled.isDefined

override def getLocalProperty(key: String): String = localProperties.getProperty(key)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -669,6 +669,16 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
assert(invocationOrder === Seq("C", "B", "A", "D"))
}

test("SPARK-46480: Add isFailed in TaskContext") {
val context = TaskContext.empty()
var isFailed = false
context.addTaskCompletionListener[Unit] { context =>
isFailed = context.isFailed()
}
context.markTaskFailed(new RuntimeException())
context.markTaskCompleted(None)
assert(isFailed)
}
}

private object TaskContextSuite {
Expand Down
4 changes: 3 additions & 1 deletion project/MimaExcludes.scala
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,9 @@ object MimaExcludes {
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.api.java.function.FlatMapGroupsWithStateFunction"),
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.api.java.function.MapGroupsWithStateFunction"),
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SaveMode"),
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.GroupState")
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.GroupState"),
// [SPARK-46480][CORE][SQL] Fix NPE when table cache task attempt
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.isFailed")
)

// Default exclude rules
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -279,9 +279,11 @@ case class CachedRDDBuilder(
cachedPlan.conf)
}
val cached = cb.mapPartitionsInternal { it =>
TaskContext.get().addTaskCompletionListener[Unit](_ => {
materializedPartitions.add(1L)
})
TaskContext.get().addTaskCompletionListener[Unit] { context =>
if (!context.isFailed() && !context.isInterrupted()) {
materializedPartitions.add(1L)
}
}
new Iterator[CachedBatch] {
override def hasNext: Boolean = it.hasNext
override def next(): CachedBatch = {
Expand Down