You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ad...@apache.org on 2015/07/17 03:14:55 UTC

spark git commit: [SPARK-8644] Include call site in SparkException stack traces thrown by job failures

Repository: spark
Updated Branches:
  refs/heads/master 031d7d414 -> 57e9b13bf


[SPARK-8644] Include call site in SparkException stack traces thrown by job failures

Example exception (new part at bottom, clearly demarcated):

```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.RuntimeException: uh-oh!
	at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38$$anonfun$apply$mcJ$sp$2.apply(DAGSchedulerSuite.scala:880)
	at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38$$anonfun$apply$mcJ$sp$2.apply(DAGSchedulerSuite.scala:880)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1640)
	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1099)
	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1099)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1777)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1777)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:63)
	at org.apache.spark.scheduler.Task.run(Task.scala:70)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:744)

Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1298)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1289)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1288)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1288)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:755)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:755)
	at scala.Option.foreach(Option.scala:236)
	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:755)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1509)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1470)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1459)
	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:560)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1744)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1762)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1777)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:1791)
	at org.apache.spark.rdd.RDD.count(RDD.scala:1099)
	at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38.apply$mcJ$sp(DAGSchedulerSuite.scala:880)
	at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38.apply(DAGSchedulerSuite.scala:880)
	at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38.apply(DAGSchedulerSuite.scala:880)
	at org.scalatest.Assertions$class.intercept(Assertions.scala:997)
	at org.scalatest.FunSuite.intercept(FunSuite.scala:1555)
	at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37.apply$mcV$sp(DAGSchedulerSuite.scala:879)
	at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37.apply(DAGSchedulerSuite.scala:878)
	at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37.apply(DAGSchedulerSuite.scala:878)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42)
	at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
	at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfter$$super$runTest(DAGSchedulerSuite.scala:70)
	at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200)
	at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(DAGSchedulerSuite.scala:70)
	at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255)
	at org.apache.spark.scheduler.DAGSchedulerSuite.runTest(DAGSchedulerSuite.scala:70)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
	at scala.collection.immutable.List.foreach(List.scala:318)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
	at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
	at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
	at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
	at org.scalatest.Suite$class.run(Suite.scala:1424)
	at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
	at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
	at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfter$$super$run(DAGSchedulerSuite.scala:70)
	at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
	at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfterAll$$super$run(DAGSchedulerSuite.scala:70)
	at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
	at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
	at org.apache.spark.scheduler.DAGSchedulerSuite.run(DAGSchedulerSuite.scala:70)
	at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
	at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
	at sbt.ForkMain$Run$2.call(ForkMain.java:294)
	at sbt.ForkMain$Run$2.call(ForkMain.java:284)
	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:744)
```

Author: Aaron Davidson <aa...@databricks.com>

Closes #7028 from aarondav/stack-trace and squashes the following commits:

4714664 [Aaron Davidson] [SPARK-8644] Include call site in SparkException stack traces thrown by job failures


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/57e9b13b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/57e9b13b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/57e9b13b

Branch: refs/heads/master
Commit: 57e9b13bf99ab82896114d47c62fbee648c9be7d
Parents: 031d7d4
Author: Aaron Davidson <aa...@databricks.com>
Authored: Thu Jul 16 18:14:45 2015 -0700
Committer: Aaron Davidson <aa...@databricks.com>
Committed: Thu Jul 16 18:14:45 2015 -0700

----------------------------------------------------------------------
 .../org/apache/spark/scheduler/DAGScheduler.scala    |  3 +++
 .../apache/spark/scheduler/DAGSchedulerSuite.scala   | 15 +++++++++++++++
 2 files changed, 18 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/57e9b13b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index f3d87ee..f8ba3d2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -556,6 +556,9 @@ class DAGScheduler(
       case JobFailed(exception: Exception) =>
         logInfo("Job %d failed: %s, took %f s".format
           (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9))
+        // SPARK-8644: Include user stack trace in exceptions coming from DAGScheduler.
+        val callerStackTrace = Thread.currentThread().getStackTrace.tail
+        exception.setStackTrace(exception.getStackTrace ++ callerStackTrace)
         throw exception
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/57e9b13b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 6bc45f2..4f2b0fa 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -875,6 +875,21 @@ class DAGSchedulerSuite
     assertDataStructuresEmpty
   }
 
+  test("Spark exceptions should include call site in stack trace") {
+    val e = intercept[SparkException] {
+      sc.parallelize(1 to 10, 2).map { _ => throw new RuntimeException("uh-oh!") }.count()
+    }
+
+    // Does not include message, ONLY stack trace.
+    val stackTraceString = e.getStackTraceString
+
+    // should actually include the RDD operation that invoked the method:
+    assert(stackTraceString.contains("org.apache.spark.rdd.RDD.count"))
+
+    // should include the FunSuite setup:
+    assert(stackTraceString.contains("org.scalatest.FunSuite"))
+  }
+
   /**
    * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations.
    * Note that this checks only the host and not the executor ID.


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org