aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/java
diff options
context:
space:
mode:
authorJosh Rosen <joshrosen@databricks.com>2015-01-14 11:45:40 -0800
committerJosh Rosen <joshrosen@databricks.com>2015-01-14 11:45:40 -0800
commit259936be710f367e1d1e019ee7b93fb68dfc33d0 (patch)
treea9637d8e3cdb9f98c748e8155efa9eaadb99437b /core/src/main/java
parent9d4449c4b3d0f5e60e71fef3a9b2e1c58a80b9e8 (diff)
downloadspark-259936be710f367e1d1e019ee7b93fb68dfc33d0.tar.gz
spark-259936be710f367e1d1e019ee7b93fb68dfc33d0.tar.bz2
spark-259936be710f367e1d1e019ee7b93fb68dfc33d0.zip
[SPARK-4014] Add TaskContext.attemptNumber and deprecate TaskContext.attemptId
`TaskContext.attemptId` is misleadingly-named, since it currently returns a taskId, which uniquely identifies a particular task attempt within a particular SparkContext, instead of an attempt number, which conveys how many times a task has been attempted. This patch deprecates `TaskContext.attemptId` and add `TaskContext.taskId` and `TaskContext.attemptNumber` fields. Prior to this change, it was impossible to determine whether a task was being re-attempted (or was a speculative copy), which made it difficult to write unit tests for tasks that fail on early attempts or speculative tasks that complete faster than original tasks. Earlier versions of the TaskContext docs suggest that `attemptId` behaves like `attemptNumber`, so there's an argument to be made in favor of changing this method's implementation. Since we've decided against making that change in maintenance branches, I think it's simpler to add better-named methods and retain the old behavior for `attemptId`; if `attemptId` behaved differently in different branches, then this would cause confusing build-breaks when backporting regression tests that rely on the new `attemptId` behavior. Most of this patch is fairly straightforward, but there is a bit of trickiness related to Mesos tasks: since there's no field in MesosTaskInfo to encode the attemptId, I packed it into the `data` field alongside the task binary. Author: Josh Rosen <joshrosen@databricks.com> Closes #3849 from JoshRosen/SPARK-4014 and squashes the following commits: 89d03e0 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 5cfff05 [Josh Rosen] Introduce wrapper for serializing Mesos task launch data. 38574d4 [Josh Rosen] attemptId -> taskAttemptId in PairRDDFunctions a180b88 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 1d43aa6 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 eee6a45 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014 0b10526 [Josh Rosen] Use putInt instead of putLong (silly mistake) 8c387ce [Josh Rosen] Use local with maxRetries instead of local-cluster. cbe4d76 [Josh Rosen] Preserve attemptId behavior and deprecate it: b2dffa3 [Josh Rosen] Address some of Reynold's minor comments 9d8d4d1 [Josh Rosen] Doc typo 1e7a933 [Josh Rosen] [SPARK-4014] Change TaskContext.attemptId to return attempt number instead of task ID. fd515a5 [Josh Rosen] Add failing test for SPARK-4014
Diffstat (limited to 'core/src/main/java')
-rw-r--r--core/src/main/java/org/apache/spark/TaskContext.java24
1 files changed, 22 insertions, 2 deletions
diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java
index 0d6973203e..095f9fb94f 100644
--- a/core/src/main/java/org/apache/spark/TaskContext.java
+++ b/core/src/main/java/org/apache/spark/TaskContext.java
@@ -62,7 +62,7 @@ public abstract class TaskContext implements Serializable {
*/
public abstract boolean isInterrupted();
- /** @deprecated: use isRunningLocally() */
+ /** @deprecated use {@link #isRunningLocally()} */
@Deprecated
public abstract boolean runningLocally();
@@ -87,19 +87,39 @@ public abstract class TaskContext implements Serializable {
* is for HadoopRDD to register a callback to close the input stream.
* Will be called in any situation - success, failure, or cancellation.
*
- * @deprecated: use addTaskCompletionListener
+ * @deprecated use {@link #addTaskCompletionListener(scala.Function1)}
*
* @param f Callback function.
*/
@Deprecated
public abstract void addOnCompleteCallback(final Function0<Unit> f);
+ /**
+ * The ID of the stage that this task belong to.
+ */
public abstract int stageId();
+ /**
+ * The ID of the RDD partition that is computed by this task.
+ */
public abstract int partitionId();
+ /**
+ * How many times this task has been attempted. The first task attempt will be assigned
+ * attemptNumber = 0, and subsequent attempts will have increasing attempt numbers.
+ */
+ public abstract int attemptNumber();
+
+ /** @deprecated use {@link #taskAttemptId()}; it was renamed to avoid ambiguity. */
+ @Deprecated
public abstract long attemptId();
+ /**
+ * An ID that is unique to this task attempt (within the same SparkContext, no two task attempts
+ * will share the same attempt ID). This is roughly equivalent to Hadoop's TaskAttemptID.
+ */
+ public abstract long taskAttemptId();
+
/** ::DeveloperApi:: */
@DeveloperApi
public abstract TaskMetrics taskMetrics();