-
Notifications
You must be signed in to change notification settings - Fork 28.9k
SPARK-3874, Provide stable TaskContext API #2782
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 all commits
ef633f5
bbd9e05
7ecc2fe
facf3b1
df261d0
ed551ce
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 |
|---|---|---|
|
|
@@ -37,68 +37,7 @@ | |
| * Contextual information about a task which can be read or mutated during execution. | ||
| */ | ||
| @DeveloperApi | ||
| public class TaskContext implements Serializable { | ||
|
|
||
| private int stageId; | ||
| private int partitionId; | ||
| private long attemptId; | ||
| private boolean runningLocally; | ||
| private TaskMetrics taskMetrics; | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * Contextual information about a task which can be read or mutated during execution. | ||
| * | ||
| * @param stageId stage id | ||
| * @param partitionId index of the partition | ||
| * @param attemptId the number of attempts to execute this task | ||
| * @param runningLocally whether the task is running locally in the driver JVM | ||
| * @param taskMetrics performance metrics of the task | ||
| */ | ||
| @DeveloperApi | ||
| public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally, | ||
| TaskMetrics taskMetrics) { | ||
| this.attemptId = attemptId; | ||
| this.partitionId = partitionId; | ||
| this.runningLocally = runningLocally; | ||
| this.stageId = stageId; | ||
| this.taskMetrics = taskMetrics; | ||
| } | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * Contextual information about a task which can be read or mutated during execution. | ||
| * | ||
| * @param stageId stage id | ||
| * @param partitionId index of the partition | ||
| * @param attemptId the number of attempts to execute this task | ||
| * @param runningLocally whether the task is running locally in the driver JVM | ||
| */ | ||
| @DeveloperApi | ||
| public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally) { | ||
| this.attemptId = attemptId; | ||
| this.partitionId = partitionId; | ||
| this.runningLocally = runningLocally; | ||
| this.stageId = stageId; | ||
| this.taskMetrics = TaskMetrics.empty(); | ||
| } | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * Contextual information about a task which can be read or mutated during execution. | ||
| * | ||
| * @param stageId stage id | ||
| * @param partitionId index of the partition | ||
| * @param attemptId the number of attempts to execute this task | ||
| */ | ||
| @DeveloperApi | ||
| public TaskContext(int stageId, int partitionId, long attemptId) { | ||
| this.attemptId = attemptId; | ||
| this.partitionId = partitionId; | ||
| this.runningLocally = false; | ||
| this.stageId = stageId; | ||
| this.taskMetrics = TaskMetrics.empty(); | ||
| } | ||
| public abstract class TaskContext implements Serializable { | ||
|
|
||
| private static ThreadLocal<TaskContext> taskContext = | ||
| new ThreadLocal<TaskContext>(); | ||
|
|
@@ -107,7 +46,7 @@ public TaskContext(int stageId, int partitionId, long attemptId) { | |
| * :: Internal API :: | ||
| * This is spark internal API, not intended to be called from user programs. | ||
| */ | ||
| public static void setTaskContext(TaskContext tc) { | ||
| static void setTaskContext(TaskContext tc) { | ||
| taskContext.set(tc); | ||
| } | ||
|
|
||
|
|
@@ -116,154 +55,60 @@ public static TaskContext get() { | |
| } | ||
|
|
||
| /** :: Internal API :: */ | ||
| public static void unset() { | ||
| static void unset() { | ||
| taskContext.remove(); | ||
| } | ||
|
|
||
| // List of callback functions to execute when the task completes. | ||
| private transient List<TaskCompletionListener> onCompleteCallbacks = | ||
| new ArrayList<TaskCompletionListener>(); | ||
|
|
||
| // Whether the corresponding task has been killed. | ||
| private volatile boolean interrupted = false; | ||
|
|
||
| // Whether the task has completed. | ||
| private volatile boolean completed = false; | ||
|
|
||
| /** | ||
| * Checks whether the task has completed. | ||
| */ | ||
| public boolean isCompleted() { | ||
| return completed; | ||
| } | ||
| public abstract boolean isCompleted(); | ||
|
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. //cc @rxin - should this be "isComplete" rather than "isCompleted"?
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. Actually this looks good - we use isCompleted elsewhere already in user-facing code.
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. My understanding is that isCompleted means it has been completed, whereas isComplete means it is "whole".
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. Yeah, I think isCompleted is right, since we'd use isFailed instead of isFailure or isFail. |
||
|
|
||
| /** | ||
| * Checks whether the task has been killed. | ||
| */ | ||
| public boolean isInterrupted() { | ||
| return interrupted; | ||
| } | ||
| public abstract boolean isInterrupted(); | ||
|
|
||
| /** | ||
| * Add a (Java friendly) listener to be executed on task completion. | ||
| * This will be called in all situation - success, failure, or cancellation. | ||
| * <p/> | ||
| * An example use is for HadoopRDD to register a callback to close the input stream. | ||
| */ | ||
| public TaskContext addTaskCompletionListener(TaskCompletionListener listener) { | ||
| onCompleteCallbacks.add(listener); | ||
| return this; | ||
| } | ||
| public abstract TaskContext addTaskCompletionListener(TaskCompletionListener listener); | ||
|
|
||
| /** | ||
| * Add a listener in the form of a Scala closure to be executed on task completion. | ||
| * This will be called in all situations - success, failure, or cancellation. | ||
| * <p/> | ||
| * An example use is for HadoopRDD to register a callback to close the input stream. | ||
| */ | ||
| public TaskContext addTaskCompletionListener(final Function1<TaskContext, Unit> f) { | ||
| onCompleteCallbacks.add(new TaskCompletionListener() { | ||
| @Override | ||
| public void onTaskCompletion(TaskContext context) { | ||
| f.apply(context); | ||
| } | ||
| }); | ||
| return this; | ||
| } | ||
| public abstract TaskContext addTaskCompletionListener(final Function1<TaskContext, Unit> f); | ||
|
|
||
| /** | ||
| * Add a callback function to be executed on task completion. An example use | ||
| * 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 | ||
| * | ||
| * | ||
| * @param f Callback function. | ||
| */ | ||
| @Deprecated | ||
| public void addOnCompleteCallback(final Function0<Unit> f) { | ||
| onCompleteCallbacks.add(new TaskCompletionListener() { | ||
| @Override | ||
| public void onTaskCompletion(TaskContext context) { | ||
| f.apply(); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| /** | ||
| * ::Internal API:: | ||
| * Marks the task as completed and triggers the listeners. | ||
| */ | ||
| public void markTaskCompleted() throws TaskCompletionListenerException { | ||
| completed = true; | ||
| List<String> errorMsgs = new ArrayList<String>(2); | ||
| // Process complete callbacks in the reverse order of registration | ||
| List<TaskCompletionListener> revlist = | ||
| new ArrayList<TaskCompletionListener>(onCompleteCallbacks); | ||
| Collections.reverse(revlist); | ||
| for (TaskCompletionListener tcl: revlist) { | ||
| try { | ||
| tcl.onTaskCompletion(this); | ||
| } catch (Throwable e) { | ||
| errorMsgs.add(e.getMessage()); | ||
| } | ||
| } | ||
|
|
||
| if (!errorMsgs.isEmpty()) { | ||
| throw new TaskCompletionListenerException(JavaConversions.asScalaBuffer(errorMsgs)); | ||
| } | ||
| } | ||
| public abstract void addOnCompleteCallback(final Function0<Unit> f); | ||
|
|
||
| /** | ||
| * ::Internal API:: | ||
| * Marks the task for interruption, i.e. cancellation. | ||
| */ | ||
| public void markInterrupted() { | ||
| interrupted = true; | ||
| } | ||
|
|
||
| @Deprecated | ||
| /** Deprecated: use getStageId() */ | ||
| public int stageId() { | ||
| return stageId; | ||
| } | ||
| public abstract int stageId(); | ||
|
|
||
| @Deprecated | ||
| /** Deprecated: use getPartitionId() */ | ||
| public int partitionId() { | ||
| return partitionId; | ||
| } | ||
| public abstract int partitionId(); | ||
|
|
||
| @Deprecated | ||
| /** Deprecated: use getAttemptId() */ | ||
| public long attemptId() { | ||
| return attemptId; | ||
| } | ||
| public abstract long attemptId(); | ||
|
|
||
| @Deprecated | ||
| /** Deprecated: use isRunningLocally() */ | ||
| public boolean runningLocally() { | ||
| return runningLocally; | ||
| } | ||
|
|
||
| public boolean isRunningLocally() { | ||
| return runningLocally; | ||
| } | ||
|
|
||
| public int getStageId() { | ||
| return stageId; | ||
| } | ||
| public abstract boolean runningLocally(); | ||
|
|
||
| public int getPartitionId() { | ||
| return partitionId; | ||
| } | ||
|
|
||
| public long getAttemptId() { | ||
| return attemptId; | ||
| } | ||
| public abstract boolean isRunningLocally(); | ||
|
|
||
| /** ::Internal API:: */ | ||
| public TaskMetrics taskMetrics() { | ||
| return taskMetrics; | ||
| } | ||
| public abstract TaskMetrics taskMetrics(); | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,26 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark | ||
|
|
||
| private [spark] object TaskContextHelper { | ||
|
||
|
|
||
| def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) | ||
|
|
||
| def unset(): Unit = TaskContext.unset() | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,91 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark | ||
|
|
||
| import org.apache.spark.executor.TaskMetrics | ||
| import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerException} | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| private[spark] class TaskContextImpl(val stageId: Int, | ||
| val partitionId: Int, | ||
| val attemptId: Long, | ||
| val runningLocally: Boolean = false, | ||
| val taskMetrics: TaskMetrics = TaskMetrics.empty) | ||
| extends TaskContext(stageId, partitionId, attemptId, runningLocally, taskMetrics) | ||
| with Logging { | ||
|
|
||
| // List of callback functions to execute when the task completes. | ||
| @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] | ||
|
|
||
| // Whether the corresponding task has been killed. | ||
| @volatile private var interrupted: Boolean = false | ||
|
|
||
| // Whether the task has completed. | ||
| @volatile private var completed: Boolean = false | ||
|
|
||
| override def addTaskCompletionListener(listener: TaskCompletionListener): this.type = { | ||
| onCompleteCallbacks += listener | ||
| this | ||
| } | ||
|
|
||
| override def addTaskCompletionListener(f: TaskContext => Unit): this.type = { | ||
| onCompleteCallbacks += new TaskCompletionListener { | ||
| override def onTaskCompletion(context: TaskContext): Unit = f(context) | ||
| } | ||
| this | ||
| } | ||
|
|
||
| @deprecated("use addTaskCompletionListener", "1.1.0") | ||
| override def addOnCompleteCallback(f: () => Unit) { | ||
| onCompleteCallbacks += new TaskCompletionListener { | ||
| override def onTaskCompletion(context: TaskContext): Unit = f() | ||
| } | ||
| } | ||
|
|
||
| /** Marks the task as completed and triggers the listeners. */ | ||
| private[spark] def markTaskCompleted(): Unit = { | ||
| completed = true | ||
| val errorMsgs = new ArrayBuffer[String](2) | ||
| // Process complete callbacks in the reverse order of registration | ||
| onCompleteCallbacks.reverse.foreach { listener => | ||
| try { | ||
| listener.onTaskCompletion(this) | ||
| } catch { | ||
| case e: Throwable => | ||
| errorMsgs += e.getMessage | ||
| logError("Error in TaskCompletionListener", e) | ||
| } | ||
| } | ||
| if (errorMsgs.nonEmpty) { | ||
| throw new TaskCompletionListenerException(errorMsgs) | ||
| } | ||
| } | ||
|
|
||
| /** Marks the task for interruption, i.e. cancellation. */ | ||
| private[spark] def markInterrupted(): Unit = { | ||
| interrupted = true | ||
| } | ||
|
|
||
| override def isCompleted: Boolean = completed | ||
|
|
||
| override def isRunningLocally: Boolean = runningLocally | ||
|
|
||
| override def isInterrupted: Boolean = interrupted | ||
| } | ||
|
|
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'd prefer this class not have a constructor (the whole idea here is that we don't want user to instantiate it). Basically we want to move all of the logic here to
TaskContextImpland only define the interface here in terms of abstract methods.