-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-22897][CORE]: Expose stageAttemptId in TaskContext #20082
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 4 commits
5753ee0
f02bc1e
59e4a9c
291bbbc
72a3abf
9266cd8
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -66,7 +66,7 @@ object TaskContext { | |
| * An empty task context that does not represent an actual task. This is only used in tests. | ||
| */ | ||
| private[spark] def empty(): TaskContextImpl = { | ||
| new TaskContextImpl(0, 0, 0, 0, null, new Properties, null) | ||
| new TaskContextImpl(0, 0, 0, 0, 0, null, new Properties, null) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -150,6 +150,11 @@ abstract class TaskContext extends Serializable { | |
| */ | ||
| def stageId(): Int | ||
|
|
||
| /** | ||
| * An ID that is unique to the stage attempt that this task belongs to. | ||
| */ | ||
| def stageAttemptId(): Int | ||
|
||
|
|
||
| /** | ||
| * The ID of the RDD partition that is computed by this task. | ||
| */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -42,6 +42,7 @@ import org.apache.spark.util._ | |
| */ | ||
| private[spark] class TaskContextImpl( | ||
| val stageId: Int, | ||
| val stageAttemptId: Int, | ||
|
||
| val partitionId: Int, | ||
| override val taskAttemptId: Long, | ||
| override val attemptNumber: Int, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,7 @@ import org.apache.spark.memory.TaskMemoryManager | |
| import org.apache.spark.metrics.source.JvmSource | ||
| import org.apache.spark.network.util.JavaUtils | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.shuffle.FetchFailedException | ||
| import org.apache.spark.util._ | ||
|
|
||
| class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { | ||
|
|
@@ -158,6 +159,30 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark | |
| assert(attemptIdsWithFailedTask.toSet === Set(0, 1)) | ||
| } | ||
|
|
||
| test("TaskContext.stageAttemptId getter") { | ||
| sc = new SparkContext("local[1,2]", "test") | ||
|
|
||
| // Check stage attemptIds are 0 for initial stage | ||
| val stageAttemptIds = sc.parallelize(Seq(1, 2), 2).mapPartitions { _ => | ||
| Seq(TaskContext.get().stageAttemptId()).iterator | ||
| }.collect() | ||
| assert(stageAttemptIds.toSet === Set(0)) | ||
|
|
||
| // Check stage attemptIds that are resubmitted when tasks have FetchFailedException | ||
| val stageAttemptIdsWithFailedStage = | ||
| sc.parallelize(Seq(1, 2, 3, 4), 4).repartition(1).mapPartitions { _ => | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You don't need |
||
| val stageAttemptId = TaskContext.get().stageAttemptId() | ||
| if (stageAttemptId < 2) { | ||
| // Throw FetchFailedException to explicitly trigger stage resubmission. A normal exception | ||
| // will only trigger task resubmission in the same stage. | ||
| throw new FetchFailedException(null, 0, 0, 0, "Fake") | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Emmm... just throw an
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Related to repartition part. I use FetchFailedException to explicitly trigger a stage resubmission. Otherwise, the task would be resubmitted in the same stage if IIRC.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. oh, right~
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add comment to explain that
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will do. |
||
| } | ||
| Seq(stageAttemptId).iterator | ||
| }.collect() | ||
|
|
||
| assert(stageAttemptIdsWithFailedStage.toSet === Set(2)) | ||
| } | ||
|
|
||
| test("accumulators are updated on exception failures") { | ||
| // This means use 1 core and 4 max task failures | ||
| sc = new SparkContext("local[1,4]", "test") | ||
|
|
@@ -190,7 +215,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark | |
| // accumulator updates from it. | ||
| val taskMetrics = TaskMetrics.empty | ||
| val task = new Task[Int](0, 0, 0) { | ||
| context = new TaskContextImpl(0, 0, 0L, 0, | ||
| context = new TaskContextImpl(0, 0, 0, 0L, 0, | ||
| new TaskMemoryManager(SparkEnv.get.memoryManager, 0L), | ||
| new Properties, | ||
| SparkEnv.get.metricsSystem, | ||
|
|
@@ -213,7 +238,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark | |
| // accumulator updates from it. | ||
| val taskMetrics = TaskMetrics.registered | ||
| val task = new Task[Int](0, 0, 0) { | ||
| context = new TaskContextImpl(0, 0, 0L, 0, | ||
| context = new TaskContextImpl(0, 0, 0, 0L, 0, | ||
| new TaskMemoryManager(SparkEnv.get.memoryManager, 0L), | ||
| new Properties, | ||
| SparkEnv.get.metricsSystem, | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we should call it
stageAttempNumberto be consistent withtaskAttemptNumber. Also let's follow the comment ofattemptNumberThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, if we are defining
stageAttemptIdfrom scratch, I would go forstageAttemptNumber. HoweverstageAttemptIdare already used elsewhere in the codebase, Like in Task.scala. I think it's more important to be consistent.However I could update the comment to reflect the attempt number part if you wish