aboutsummaryrefslogtreecommitdiff
path: root/core/src/test
diff options
context:
space:
mode:
authorReynold Xin <rxin@databricks.com>2016-02-26 12:49:16 -0800
committerDavies Liu <davies.liu@gmail.com>2016-02-26 12:49:16 -0800
commit391755dc6ed2e156b8df8a530ac8df6ed7ba7f8a (patch)
tree7ebb73ea7e90c5a4c180f493c8d8bdaffbf2d8f2 /core/src/test
parent0598a2b81d1426dd2cf9e6fc32cef345364d18c6 (diff)
downloadspark-391755dc6ed2e156b8df8a530ac8df6ed7ba7f8a.tar.gz
spark-391755dc6ed2e156b8df8a530ac8df6ed7ba7f8a.tar.bz2
spark-391755dc6ed2e156b8df8a530ac8df6ed7ba7f8a.zip
[SPARK-13465] Add a task failure listener to TaskContext
## What changes were proposed in this pull request? TaskContext supports task completion callback, which gets called regardless of task failures. However, there is no way for the listener to know if there is an error. This patch adds a new listener that gets called when a task fails. ## How was the this patch tested? New unit test case and integration test case covering the code path Author: Reynold Xin <rxin@databricks.com> Closes #11340 from rxin/SPARK-13465.
Diffstat (limited to 'core/src/test')
-rw-r--r--core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java39
-rw-r--r--core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java30
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala44
3 files changed, 73 insertions, 40 deletions
diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java b/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java
deleted file mode 100644
index e38bc38949..0000000000
--- a/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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;
-import org.apache.spark.util.TaskCompletionListener;
-
-
-/**
- * A simple implementation of TaskCompletionListener that makes sure TaskCompletionListener and
- * TaskContext is Java friendly.
- */
-public class JavaTaskCompletionListenerImpl implements TaskCompletionListener {
-
- @Override
- public void onTaskCompletion(TaskContext context) {
- context.isCompleted();
- context.isInterrupted();
- context.stageId();
- context.partitionId();
- context.isRunningLocally();
- context.addTaskCompletionListener(this);
- }
-}
diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java
index 4a918f725d..f914081d7d 100644
--- a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java
+++ b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java
@@ -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.
@@ -32,10 +34,38 @@ public class JavaTaskContextCompileCheck {
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) {
+ }
+ }
+
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
index 850e470ca1..c4cf2f9f70 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
@@ -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 {
@@ -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])
@@ -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
@@ -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