Skip to content
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

Include stack trace for exceptions thrown by user code. #409

Closed
wants to merge 2 commits into from

Conversation

marmbrus
Copy link
Contributor

It is very confusing when your code throws an exception, but the only stack trace show is in the DAGScheduler. This is a simple patch to include the stack trace for the actual failure in the error message. Suggestions on formatting welcome.

Before:

scala> sc.parallelize(1 :: Nil).map(_ => sys.error("Ahh!")).collect()
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0.0:3 failed 1 times (most recent failure: Exception failure in TID 3 on host localhost: java.lang.RuntimeException: Ahh!)
    at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1055)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1039)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1037)
...

After:

org.apache.spark.SparkException: Job aborted due to stage failure: Task 0.0:3 failed 1 times, most recent failure: Exception failure in TID 3 on host localhost: java.lang.RuntimeException: Ahh!
        scala.sys.package$.error(package.scala:27)
        $iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:13)
        $iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:13)
        scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
        scala.collection.Iterator$class.foreach(Iterator.scala:727)
        scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
        scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
        scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
        scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
        scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
        scala.collection.AbstractIterator.to(Iterator.scala:1157)
        scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
        scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
        scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
        scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
        org.apache.spark.rdd.RDD$$anonfun$6.apply(RDD.scala:676)
        org.apache.spark.rdd.RDD$$anonfun$6.apply(RDD.scala:676)
        org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1048)
        org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1048)
        org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:110)
        org.apache.spark.scheduler.Task.run(Task.scala:50)
        org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$1.apply$mcV$sp(Executor.scala:211)
        org.apache.spark.deploy.SparkHadoopUtil.runAsUser(SparkHadoopUtil.scala:46)
        org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:176)
        java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        java.lang.Thread.run(Thread.java:744)
Driver stacktrace:
    at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1055)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1039)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1037)
    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.org$apache$spark$scheduler$DAGScheduler$$abortStage(DAGScheduler.scala:1037)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$processEvent$10.apply(DAGScheduler.scala:614)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$processEvent$10.apply(DAGScheduler.scala:614)
    at scala.Option.foreach(Option.scala:236)
    at org.apache.spark.scheduler.DAGScheduler.processEvent(DAGScheduler.scala:614)
    at org.apache.spark.scheduler.DAGScheduler$$anonfun$start$1$$anon$2$$anonfun$receive$1.applyOrElse(DAGScheduler.scala:143)
    at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498)
    at akka.actor.ActorCell.invoke(ActorCell.scala:456)
    at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237)
    at akka.dispatch.Mailbox.run(Mailbox.scala:219)
    at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
    at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
    at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
    at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
    at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished.

@AmplabJenkins
Copy link

Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14131/

@marmbrus
Copy link
Contributor Author

Updated with suggestions from @aarondav.

@AmplabJenkins
Copy link

Merged build triggered.

@AmplabJenkins
Copy link

Merged build started.

@AmplabJenkins
Copy link

Merged build finished. All automated tests passed.

@AmplabJenkins
Copy link

All automated tests passed.
Refer to this link for build results: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/14138/

@rxin
Copy link
Contributor

rxin commented Apr 17, 2014

thanks. merged.

@asfgit asfgit closed this in d4916a8 Apr 17, 2014
asfgit pushed a commit that referenced this pull request Apr 17, 2014
It is very confusing when your code throws an exception, but the only stack trace show is in the DAGScheduler.  This is a simple patch to include the stack trace for the actual failure in the error message.  Suggestions on formatting welcome.

Before:
```
scala> sc.parallelize(1 :: Nil).map(_ => sys.error("Ahh!")).collect()
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0.0:3 failed 1 times (most recent failure: Exception failure in TID 3 on host localhost: java.lang.RuntimeException: Ahh!)
	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1055)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1039)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1037)
...
```

After:
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0.0:3 failed 1 times, most recent failure: Exception failure in TID 3 on host localhost: java.lang.RuntimeException: Ahh!
        scala.sys.package$.error(package.scala:27)
        $iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:13)
        $iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:13)
        scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
        scala.collection.Iterator$class.foreach(Iterator.scala:727)
        scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
        scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
        scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
        scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
        scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
        scala.collection.AbstractIterator.to(Iterator.scala:1157)
        scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
        scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
        scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
        scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
        org.apache.spark.rdd.RDD$$anonfun$6.apply(RDD.scala:676)
        org.apache.spark.rdd.RDD$$anonfun$6.apply(RDD.scala:676)
        org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1048)
        org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1048)
        org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:110)
        org.apache.spark.scheduler.Task.run(Task.scala:50)
        org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$1.apply$mcV$sp(Executor.scala:211)
        org.apache.spark.deploy.SparkHadoopUtil.runAsUser(SparkHadoopUtil.scala:46)
        org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:176)
        java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        java.lang.Thread.run(Thread.java:744)
Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1055)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1039)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1037)
	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.org$apache$spark$scheduler$DAGScheduler$$abortStage(DAGScheduler.scala:1037)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$processEvent$10.apply(DAGScheduler.scala:614)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$processEvent$10.apply(DAGScheduler.scala:614)
	at scala.Option.foreach(Option.scala:236)
	at org.apache.spark.scheduler.DAGScheduler.processEvent(DAGScheduler.scala:614)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$start$1$$anon$2$$anonfun$receive$1.applyOrElse(DAGScheduler.scala:143)
	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498)
	at akka.actor.ActorCell.invoke(ActorCell.scala:456)
	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237)
	at akka.dispatch.Mailbox.run(Mailbox.scala:219)
	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
```

Author: Michael Armbrust <[email protected]>

Closes #409 from marmbrus/stacktraces and squashes the following commits:

3e4eb65 [Michael Armbrust] indent. include header for driver stack trace.
018b06b [Michael Armbrust] Include stack trace for exceptions in user code.

(cherry picked from commit d4916a8)
Signed-off-by: Reynold Xin <[email protected]>
@marmbrus marmbrus deleted the stacktraces branch April 23, 2014 22:01
pwendell added a commit to pwendell/spark that referenced this pull request May 12, 2014
Automatically unpersisting RDDs that have been cleaned up from DStreams

Earlier RDDs generated by DStreams were forgotten but not unpersisted. The system relied on the natural BlockManager LRU to drop the data. The cleaner.ttl was a hammer to clean up RDDs but it is something that needs to be set separately and need to be set very conservatively (at best, few minutes). This automatic unpersisting allows the system to handle this automatically, which reduces memory usage. As a side effect it will also improve GC performance as there are less number of objects stored in memory. In fact, for some workloads, it may allow RDDs to be cached as deserialized, which speeds up processing without too much GC overheads.

This is disabled by default. To enable it set configuration spark.streaming.unpersist to true. In future release, this will be set to true by default.

Also, reduced sleep time in TaskSchedulerImpl.stop() from 5 second to 1 second. From my conversation with Matei, there does not seem to be any good reason for the sleep for letting messages be sent out be so long.
pdeyhim pushed a commit to pdeyhim/spark-1 that referenced this pull request Jun 25, 2014
It is very confusing when your code throws an exception, but the only stack trace show is in the DAGScheduler.  This is a simple patch to include the stack trace for the actual failure in the error message.  Suggestions on formatting welcome.

Before:
```
scala> sc.parallelize(1 :: Nil).map(_ => sys.error("Ahh!")).collect()
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0.0:3 failed 1 times (most recent failure: Exception failure in TID 3 on host localhost: java.lang.RuntimeException: Ahh!)
	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1055)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1039)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1037)
...
```

After:
```
org.apache.spark.SparkException: Job aborted due to stage failure: Task 0.0:3 failed 1 times, most recent failure: Exception failure in TID 3 on host localhost: java.lang.RuntimeException: Ahh!
        scala.sys.package$.error(package.scala:27)
        $iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:13)
        $iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:13)
        scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
        scala.collection.Iterator$class.foreach(Iterator.scala:727)
        scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
        scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
        scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
        scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
        scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
        scala.collection.AbstractIterator.to(Iterator.scala:1157)
        scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
        scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
        scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
        scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
        org.apache.spark.rdd.RDD$$anonfun$6.apply(RDD.scala:676)
        org.apache.spark.rdd.RDD$$anonfun$6.apply(RDD.scala:676)
        org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1048)
        org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1048)
        org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:110)
        org.apache.spark.scheduler.Task.run(Task.scala:50)
        org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$1.apply$mcV$sp(Executor.scala:211)
        org.apache.spark.deploy.SparkHadoopUtil.runAsUser(SparkHadoopUtil.scala:46)
        org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:176)
        java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        java.lang.Thread.run(Thread.java:744)
Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1055)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1039)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$org$apache$spark$scheduler$DAGScheduler$$abortStage$1.apply(DAGScheduler.scala:1037)
	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.org$apache$spark$scheduler$DAGScheduler$$abortStage(DAGScheduler.scala:1037)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$processEvent$10.apply(DAGScheduler.scala:614)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$processEvent$10.apply(DAGScheduler.scala:614)
	at scala.Option.foreach(Option.scala:236)
	at org.apache.spark.scheduler.DAGScheduler.processEvent(DAGScheduler.scala:614)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$start$1$$anon$2$$anonfun$receive$1.applyOrElse(DAGScheduler.scala:143)
	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498)
	at akka.actor.ActorCell.invoke(ActorCell.scala:456)
	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237)
	at akka.dispatch.Mailbox.run(Mailbox.scala:219)
	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
```

Author: Michael Armbrust <[email protected]>

Closes apache#409 from marmbrus/stacktraces and squashes the following commits:

3e4eb65 [Michael Armbrust] indent. include header for driver stack trace.
018b06b [Michael Armbrust] Include stack trace for exceptions in user code.
mccheah pushed a commit to mccheah/spark that referenced this pull request Nov 28, 2018
bzhaoopenstack pushed a commit to bzhaoopenstack/spark that referenced this pull request Sep 11, 2019
Change to run Packer jobs on v1.3.3
arjunshroff pushed a commit to arjunshroff/spark that referenced this pull request Nov 24, 2020
RolatZhang pushed a commit to RolatZhang/spark that referenced this pull request Mar 18, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants