Skip to content

Commit

Permalink
Update upstream
Browse files Browse the repository at this point in the history
  • Loading branch information
HyukjinKwon committed Feb 29, 2016
2 parents c703def + cca79fa commit 8efb0e3
Show file tree
Hide file tree
Showing 135 changed files with 2,265 additions and 1,465 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@
*
* This implementation is largely based on the {@code CountMinSketch} class from stream-lib.
*/
abstract public class CountMinSketch {
public abstract class CountMinSketch {

public enum Version {
/**
Expand Down
28 changes: 26 additions & 2 deletions core/src/main/scala/org/apache/spark/TaskContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.memory.TaskMemoryManager
import org.apache.spark.metrics.source.Source
import org.apache.spark.util.TaskCompletionListener
import org.apache.spark.util.{TaskCompletionListener, TaskFailureListener}


object TaskContext {
Expand Down Expand Up @@ -106,15 +106,39 @@ abstract class TaskContext extends Serializable {
* 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.
*
* Exceptions thrown by the listener will result in failure of the task.
*/
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.
*
* Exceptions thrown by the listener will result in failure of the task.
*/
def addTaskCompletionListener(f: (TaskContext) => Unit): TaskContext
def addTaskCompletionListener(f: (TaskContext) => Unit): TaskContext = {
addTaskCompletionListener(new TaskCompletionListener {
override def onTaskCompletion(context: TaskContext): Unit = f(context)
})
}

/**
* Adds a listener to be executed on task failure.
* Operations defined here must be idempotent, as `onTaskFailure` can be called multiple times.
*/
def addTaskFailureListener(listener: TaskFailureListener): TaskContext

/**
* Adds a listener to be executed on task failure.
* Operations defined here must be idempotent, as `onTaskFailure` can be called multiple times.
*/
def addTaskFailureListener(f: (TaskContext, Throwable) => Unit): TaskContext = {
addTaskFailureListener(new TaskFailureListener {
override def onTaskFailure(context: TaskContext, error: Throwable): Unit = f(context, error)
})
}

/**
* The ID of the stage that this task belong to.
Expand Down
33 changes: 26 additions & 7 deletions core/src/main/scala/org/apache/spark/TaskContextImpl.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import org.apache.spark.executor.TaskMetrics
import org.apache.spark.memory.TaskMemoryManager
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.metrics.source.Source
import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerException}
import org.apache.spark.util._

private[spark] class TaskContextImpl(
val stageId: Int,
Expand All @@ -41,9 +41,12 @@ private[spark] class TaskContextImpl(
*/
override val taskMetrics: TaskMetrics = new TaskMetrics(initialAccumulators)

// List of callback functions to execute when the task completes.
/** List of callback functions to execute when the task completes. */
@transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener]

/** List of callback functions to execute when the task fails. */
@transient private val onFailureCallbacks = new ArrayBuffer[TaskFailureListener]

// Whether the corresponding task has been killed.
@volatile private var interrupted: Boolean = false

Expand All @@ -55,14 +58,30 @@ private[spark] class TaskContextImpl(
this
}

override def addTaskCompletionListener(f: TaskContext => Unit): this.type = {
onCompleteCallbacks += new TaskCompletionListener {
override def onTaskCompletion(context: TaskContext): Unit = f(context)
}
override def addTaskFailureListener(listener: TaskFailureListener): this.type = {
onFailureCallbacks += listener
this
}

/** Marks the task as completed and triggers the listeners. */
/** Marks the task as completed and triggers the failure listeners. */
private[spark] def markTaskFailed(error: Throwable): Unit = {
val errorMsgs = new ArrayBuffer[String](2)
// Process complete callbacks in the reverse order of registration
onFailureCallbacks.reverse.foreach { listener =>
try {
listener.onTaskFailure(this, error)
} catch {
case e: Throwable =>
errorMsgs += e.getMessage
logError("Error in TaskFailureListener", e)
}
}
if (errorMsgs.nonEmpty) {
throw new TaskCompletionListenerException(errorMsgs, Option(error))
}
}

/** Marks the task as completed and triggers the completion listeners. */
private[spark] def markTaskCompleted(): Unit = {
completed = true
val errorMsgs = new ArrayBuffer[String](2)
Expand Down
5 changes: 5 additions & 0 deletions core/src/main/scala/org/apache/spark/scheduler/Task.scala
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,12 @@ private[spark] abstract class Task[T](
}
try {
runTask(context)
} catch { case e: Throwable =>
// Catch all errors; run task failure callbacks, and rethrow the exception.
context.markTaskFailed(e)
throw e
} finally {
// Call the task completion callbacks.
context.markTaskCompleted()
try {
Utils.tryLogNonFatalError {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,6 +179,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
context.reply(true)

case RemoveExecutor(executorId, reason) =>
// We will remove the executor's state and cannot restore it. However, the connection
// between the driver and the executor may be still alive so that the executor won't exit
// automatically, so try to tell the executor to stop itself. See SPARK-13519.
executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor))
removeExecutor(executorId, reason)
context.reply(true)

Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -29,5 +29,40 @@ import org.apache.spark.annotation.DeveloperApi
*/
@DeveloperApi
trait TaskCompletionListener extends EventListener {
def onTaskCompletion(context: TaskContext)
def onTaskCompletion(context: TaskContext): Unit
}


/**
* :: DeveloperApi ::
*
* Listener providing a callback function to invoke when a task's execution encounters an error.
* Operations defined here must be idempotent, as `onTaskFailure` can be called multiple times.
*/
@DeveloperApi
trait TaskFailureListener extends EventListener {
def onTaskFailure(context: TaskContext, error: Throwable): Unit
}


/**
* Exception thrown when there is an exception in executing the callback in TaskCompletionListener.
*/
private[spark]
class TaskCompletionListenerException(
errorMessages: Seq[String],
val previousError: Option[Throwable] = None)
extends RuntimeException {

override def getMessage: String = {
if (errorMessages.size == 1) {
errorMessages.head
} else {
errorMessages.zipWithIndex.map { case (msg, i) => s"Exception $i: $msg" }.mkString("\n")
} +
previousError.map { e =>
"\n\nPrevious exception in task: " + e.getMessage + "\n" +
e.getStackTrace.mkString("\t", "\n\t", "")
}.getOrElse("")
}
}

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@
package test.org.apache.spark;

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

/**
* Something to make sure that TaskContext can be used in Java.
Expand All @@ -32,10 +34,38 @@ public static void test() {
tc.isRunningLocally();

tc.addTaskCompletionListener(new JavaTaskCompletionListenerImpl());
tc.addTaskFailureListener(new JavaTaskFailureListenerImpl());

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

/**
* A simple implementation of TaskCompletionListener that makes sure TaskCompletionListener and
* TaskContext is Java friendly.
*/
static class JavaTaskCompletionListenerImpl implements TaskCompletionListener {
@Override
public void onTaskCompletion(TaskContext context) {
context.isCompleted();
context.isInterrupted();
context.stageId();
context.partitionId();
context.isRunningLocally();
context.addTaskCompletionListener(this);
}
}

/**
* A simple implementation of TaskCompletionListener that makes sure TaskCompletionListener and
* TaskContext is Java friendly.
*/
static class JavaTaskFailureListenerImpl implements TaskFailureListener {
@Override
public void onTaskFailure(TaskContext context, Throwable error) {
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,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.util.{TaskCompletionListener, TaskCompletionListenerException}
import org.apache.spark.util._

class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext {

Expand Down Expand Up @@ -66,6 +66,26 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
assert(TaskContextSuite.completed === true)
}

test("calls TaskFailureListeners after failure") {
TaskContextSuite.lastError = null
sc = new SparkContext("local", "test")
val rdd = new RDD[String](sc, List()) {
override def getPartitions = Array[Partition](StubPartition(0))
override def compute(split: Partition, context: TaskContext) = {
context.addTaskFailureListener((context, error) => TaskContextSuite.lastError = error)
sys.error("damn error")
}
}
val closureSerializer = SparkEnv.get.closureSerializer.newInstance()
val func = (c: TaskContext, i: Iterator[String]) => i.next()
val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func))))
val task = new ResultTask[String, String](0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0)
intercept[RuntimeException] {
task.run(0, 0, null)
}
assert(TaskContextSuite.lastError.getMessage == "damn error")
}

test("all TaskCompletionListeners should be called even if some fail") {
val context = TaskContext.empty()
val listener = mock(classOf[TaskCompletionListener])
Expand All @@ -80,6 +100,26 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark
verify(listener, times(1)).onTaskCompletion(any())
}

test("all TaskFailureListeners should be called even if some fail") {
val context = TaskContext.empty()
val listener = mock(classOf[TaskFailureListener])
context.addTaskFailureListener((_, _) => throw new Exception("exception in listener1"))
context.addTaskFailureListener(listener)
context.addTaskFailureListener((_, _) => throw new Exception("exception in listener3"))

val e = intercept[TaskCompletionListenerException] {
context.markTaskFailed(new Exception("exception in task"))
}

// Make sure listener 2 was called.
verify(listener, times(1)).onTaskFailure(any(), any())

// also need to check failure in TaskFailureListener does not mask earlier exception
assert(e.getMessage.contains("exception in listener1"))
assert(e.getMessage.contains("exception in listener3"))
assert(e.getMessage.contains("exception in task"))
}

test("TaskContext.attemptNumber should return attempt number, not task id (SPARK-4014)") {
sc = new SparkContext("local[1,2]", "test") // use maxRetries = 2 because we test failed tasks
// Check that attemptIds are 0 for all tasks' initial attempts
Expand Down Expand Up @@ -153,6 +193,8 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark

private object TaskContextSuite {
@volatile var completed = false

@volatile var lastError: Throwable = _
}

private case class StubPartition(index: Int) extends Partition
Loading

0 comments on commit 8efb0e3

Please sign in to comment.