-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-3543] Write TaskContext in Java and expose it through a static accessor. #2425
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
333c7d6
f716fd1
edf945e
a7d5e23
ddb8cbe
ee8bd00
8ae414c
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 |
|---|---|---|
| @@ -0,0 +1,234 @@ | ||
| /* | ||
| * 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 java.io.Serializable; | ||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
|
|
||
| import scala.Function0; | ||
| import scala.Function1; | ||
| import scala.Unit; | ||
| import scala.collection.JavaConversions; | ||
|
|
||
| import org.apache.spark.annotation.DeveloperApi; | ||
| import org.apache.spark.executor.TaskMetrics; | ||
| import org.apache.spark.util.TaskCompletionListener; | ||
| import org.apache.spark.util.TaskCompletionListenerException; | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
| * Contextual information about a task which can be read or mutated during execution. | ||
| */ | ||
| @DeveloperApi | ||
| public class TaskContext implements Serializable { | ||
|
|
||
| private Integer stageId; | ||
| private Integer 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(Integer stageId, Integer partitionId, Long attemptId, Boolean runningLocally, | ||
|
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. fyi all these should be primitives rather than boxed Integers/Longs/Booleans. I'm going to fix it in a separate pr. |
||
| TaskMetrics taskMetrics) { | ||
| this.attemptId = attemptId; | ||
| this.partitionId = partitionId; | ||
| this.runningLocally = runningLocally; | ||
| this.stageId = stageId; | ||
| this.taskMetrics = taskMetrics; | ||
| taskContext.set(this); | ||
|
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. I think it might be better to explicitly set this in the executor when we create the context rather than do it as side effect of construction. What do you think @rxin? |
||
| } | ||
|
|
||
|
|
||
| /** | ||
| * :: 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(Integer stageId, Integer partitionId, Long attemptId, | ||
| Boolean runningLocally) { | ||
| this.attemptId = attemptId; | ||
| this.partitionId = partitionId; | ||
| this.runningLocally = runningLocally; | ||
| this.stageId = stageId; | ||
| this.taskMetrics = TaskMetrics.empty(); | ||
| taskContext.set(this); | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * :: 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(Integer stageId, Integer partitionId, Long attemptId) { | ||
| this.attemptId = attemptId; | ||
| this.partitionId = partitionId; | ||
| this.runningLocally = false; | ||
| this.stageId = stageId; | ||
| this.taskMetrics = TaskMetrics.empty(); | ||
| taskContext.set(this); | ||
| } | ||
|
|
||
| private static ThreadLocal<TaskContext> taskContext = | ||
|
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. I would remove this and the below function. I think it's better if a
Member
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. But then its already static code ? + Where else should I put it ? I
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. I think Patrick meant putting it in executor or in Task's run method
Member
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. alright
Member
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. But then how would someone access it ? I hope he wants to retain atleast the get method inside TaskContext as static.
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. yes, get must be static. he's only thinking we should move the set to somewhere else
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. assuming i'm reading his mind correctly :) |
||
| new ThreadLocal<TaskContext>(); | ||
|
|
||
| public static TaskContext get() { | ||
| return taskContext.get(); | ||
| } | ||
|
|
||
| // 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; | ||
| } | ||
|
|
||
| /** | ||
| * Checks whether the task has been killed. | ||
| */ | ||
| public Boolean isInterrupted() { | ||
| return interrupted; | ||
| } | ||
|
|
||
| /** | ||
| * 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; | ||
| } | ||
|
|
||
| /** | ||
| * Add a listener in the form of a Scala closure to be executed on task completion. | ||
| * This will be called in all situation - success, failure, or cancellation. | ||
|
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. all situations |
||
| * <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; | ||
| } | ||
|
|
||
| /** | ||
| * 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. | ||
| * | ||
| * @param f Callback function. | ||
| */ | ||
| @Deprecated | ||
| public void addOnCompleteCallback(final Function0<Unit> f) { | ||
|
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. We don't have the deprecation instructions any more - can you add them to the Javadoc? We should direct users to addTaskCompletionListener |
||
| onCompleteCallbacks.add(new TaskCompletionListener() { | ||
| @Override | ||
| public void onTaskCompletion(TaskContext context) { | ||
| f.apply(); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| /** | ||
| * 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) { | ||
|
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.
|
||
| try { | ||
| tcl.onTaskCompletion(this); | ||
| } catch (Throwable e) { | ||
| errorMsgs.add(e.getMessage()); | ||
| } | ||
| } | ||
|
|
||
| if (!errorMsgs.isEmpty()) { | ||
| throw new TaskCompletionListenerException(JavaConversions.asScalaBuffer(errorMsgs)); | ||
| } | ||
| taskContext.remove(); | ||
| } | ||
|
|
||
| /** | ||
| * Marks the task for interruption, i.e. cancellation. | ||
| */ | ||
| public void markInterrupted() { | ||
| interrupted = true; | ||
| } | ||
|
|
||
| public Integer stageId() { | ||
|
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. int? why box? |
||
| return stageId; | ||
| } | ||
|
|
||
| public Integer partitionId() { | ||
| return partitionId; | ||
| } | ||
|
|
||
| public Long attemptId() { | ||
|
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. long |
||
| return attemptId; | ||
| } | ||
|
|
||
| public Boolean runningLocally() { | ||
|
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. bool |
||
| return runningLocally; | ||
| } | ||
|
|
||
| public TaskMetrics taskMetrics() { | ||
|
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. can u try remove the public modifier and see if it still compiles? I think in that case it defaults to package private, which is the same as the Scala one
Member
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. It doesn't, because java is too strict about package private. The package name should be exact same does not expose it even in sub-packages.
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. ok then we should document clearly that it is not a public api |
||
| return taskMetrics; | ||
| } | ||
| } | ||
This file was deleted.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -94,7 +94,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar | |
| } | ||
|
|
||
| whenExecuting(blockManager) { | ||
| val context = new TaskContext(0, 0, 0, runningLocally = true) | ||
| val context = new TaskContext(0, 0, 0, true) | ||
|
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. we should name it here too
Member
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. These were breaking compilations.. |
||
| val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) | ||
| assert(value.toList === List(1, 2, 3, 4)) | ||
| } | ||
|
|
||
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.
int, int, long, bool