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
126 changes: 0 additions & 126 deletions core/src/main/java/org/apache/spark/TaskContext.java

This file was deleted.

136 changes: 136 additions & 0 deletions core/src/main/scala/org/apache/spark/TaskContext.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
/*
* 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 org.apache.spark.annotation.DeveloperApi
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.util.TaskCompletionListener


object TaskContext {
/**
* Return the currently active TaskContext. This can be called inside of
* user functions to access contextual information about running tasks.
*/
def get(): TaskContext = taskContext.get

private val taskContext: ThreadLocal[TaskContext] = new ThreadLocal[TaskContext]

// Note: protected[spark] instead of private[spark] to prevent the following two from
// showing up in JavaDoc.
/**
* Set the thread local TaskContext. Internal to Spark.
*/
protected[spark] def setTaskContext(tc: TaskContext): Unit = taskContext.set(tc)

/**
* Unset the thread local TaskContext. Internal to Spark.
*/
protected[spark] def unset(): Unit = taskContext.remove()
}


/**
* Contextual information about a task which can be read or mutated during
* execution. To access the TaskContext for a running task, use:
* {{{
* org.apache.spark.TaskContext.get()
* }}}
*/
abstract class TaskContext extends Serializable {
// Note: TaskContext must NOT define a get method. Otherwise it will prevent the Scala compiler
// from generating a static get method (based on the companion object's get method).

// Note: Update JavaTaskContextCompileCheck when new methods are added to this class.

// Note: getters in this class are defined with parentheses to maintain backward compatibility.

/**
* Returns true if the task has completed.
*/
def isCompleted(): Boolean

/**
* Returns true if the task has been killed.
*/
def isInterrupted(): Boolean

@deprecated("1.2.0", "use isRunningLocally")
def runningLocally(): Boolean

/**
* Returns true if the task is running locally in the driver program.
* @return
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can probably drop this @return tag.

*/
def isRunningLocally(): Boolean

/**
* Adds a (Java friendly) listener to be executed on task completion.
* This will be called in all situation - success, failure, or cancellation.
* An example use is for HadoopRDD to register a callback to close the input stream.
*/
def addTaskCompletionListener(listener: TaskCompletionListener): TaskContext

/**
* Adds 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.
* An example use is for HadoopRDD to register a callback to close the input stream.
*/
def addTaskCompletionListener(f: (TaskContext) => Unit): TaskContext

/**
* Adds 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("1.2.0", "use addTaskCompletionListener")
def addOnCompleteCallback(f: () => Unit)

/**
* The ID of the stage that this task belong to.
*/
def stageId(): Int

/**
* The ID of the RDD partition that is computed by this task.
*/
def partitionId(): Int

/**
* 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.
*/
def attemptNumber(): Int

@deprecated("1.3.0", "use attemptNumber")
def attemptId(): Long

/**
* 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.
*/
def taskAttemptId(): Long

/** ::DeveloperApi:: */
@DeveloperApi
def taskMetrics(): TaskMetrics
}
8 changes: 4 additions & 4 deletions core/src/main/scala/org/apache/spark/TaskContextImpl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ private[spark] class TaskContextImpl(
with Logging {

// For backwards-compatibility; this method is now deprecated as of 1.3.0.
override def attemptId: Long = taskAttemptId
override def attemptId(): Long = taskAttemptId

// List of callback functions to execute when the task completes.
@transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener]
Expand Down Expand Up @@ -87,10 +87,10 @@ private[spark] class TaskContextImpl(
interrupted = true
}

override def isCompleted: Boolean = completed
override def isCompleted(): Boolean = completed

override def isRunningLocally: Boolean = runningLocally
override def isRunningLocally(): Boolean = runningLocally

override def isInterrupted: Boolean = interrupted
override def isInterrupted(): Boolean = interrupted
}

Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,10 @@
* limitations under the License.
*/

package org.apache.spark.util;
package test.org.apache.spark;
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

note that I changed the package to make sure we test package visibility correctly also.

We should do this for other Java API tests, but we can do those later.


import org.apache.spark.TaskContext;
import org.apache.spark.util.TaskCompletionListener;


/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,41 @@
/*
* 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 test.org.apache.spark;

import org.apache.spark.TaskContext;

/**
* Something to make sure that TaskContext can be used in Java.
*/
public class JavaTaskContextCompileCheck {

public static void test() {
TaskContext tc = TaskContext.get();

tc.isCompleted();
tc.isInterrupted();
tc.isRunningLocally();

tc.addTaskCompletionListener(new JavaTaskCompletionListenerImpl());

tc.attemptNumber();
tc.partitionId();
tc.stageId();
tc.taskAttemptId();
}
}