You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zeppelin.apache.org by mo...@apache.org on 2015/05/13 01:49:24 UTC

incubator-zeppelin git commit: [ZEPPELIN-75]: PySpark interpreter - useful debugging traceback information is lost for any error from Spark

Repository: incubator-zeppelin
Updated Branches:
  refs/heads/master a49184cd3 -> aaee71977


[ZEPPELIN-75]: PySpark interpreter - useful debugging traceback information is lost for any error from Spark

When there is an error from Spark, the original error is not returned as output in the cell, instead a generic Py4JError is shown.

While it is possible to look at zeppelin-interpreter-spark-root-node.log, it might not be accessible for a multi-user environment as it will require remote access to the host running Zeppelin.

Before:

```
(<class 'py4j.protocol.Py4JJavaError'>, Py4JJavaError(u'An error occurred while calling o45.collect.\n', JavaObject id=o46), <traceback object at 0x7fb737ea72d8>)
```

Almost all error from Spark will look like this with Py4JJavaError.

After:

```
Py4JJavaError: An error occurred while calling o45.collect.
: 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): org.apache.spark.api.python.PythonException: Traceback (most recent call last):
  File "/opt/spark-1.3.0-bin-hadoop2.4/python/pyspark/worker.py", line 101, in main
    process()
  File "/opt/spark-1.3.0-bin-hadoop2.4/python/pyspark/worker.py", line 96, in process
    serializer.dump_stream(func(split_index, iterator), outfile)
  File "/opt/spark-1.3.0-bin-hadoop2.4/python/pyspark/serializers.py", line 236, in dump_stream
    vs = list(itertools.islice(iterator, batch))
  File "/opt/spark-1.3.0-bin-hadoop2.4/python/pyspark/rdd.py", line 735, in func
    initial = next(iterator)
  File "<string>", line 2, in sample
TypeError: 'module' object is not callable

	at org.apache.spark.api.python.PythonRDD$$anon$1.read(PythonRDD.scala:135)
	at org.apache.spark.api.python.PythonRDD$$anon$1.<init>(PythonRDD.scala:176)
	at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:94)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:244)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
	at org.apache.spark.scheduler.Task.run(Task.scala:64)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:203)
	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:745)

Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1203)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1192)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1191)
	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:1191)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:693)
	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:693)
	at scala.Option.foreach(Option.scala:236)
	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:693)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1393)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1354)
	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)

(<class 'py4j.protocol.Py4JJavaError'>, Py4JJavaError(u'An error occurred while calling o45.collect.\n', JavaObject id=o46), <traceback object at 0x7f8b45deb3f8>)
```

Author: Felix Cheung <fe...@hotmail.com>

Closes #66 from felixcheung/master and squashes the following commits:

c475497 [Felix Cheung] Merge commit '956e3f74a1b2f28fd8caa25055e77f687ca8d211'
a55666d [Felix Cheung] PySpark error handling improvement - return more meaningful message from the original error which is useful for Spark related errors
dfbb458 [Felix Cheung] Merge commit 'e23f3034053fbd8b8f4eff478c372d151a42c36b'
e89ba08 [Felix Cheung] PySpark Interpreter should allow starting with a specific version of Python, as PySpark does.
65ba046 [Felix Cheung] Merge commit 'a007a9b5f235ebd9c608a005c5243503291d94d5'
7a30a14 [Felix Cheung] minor doc update for running on YARN


Project: http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/commit/aaee7197
Tree: http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/tree/aaee7197
Diff: http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/diff/aaee7197

Branch: refs/heads/master
Commit: aaee7197754fb4da8ec7b1241a4b9aa7ba362312
Parents: a49184c
Author: Felix Cheung <fe...@hotmail.com>
Authored: Sat May 9 22:01:01 2015 -0700
Committer: Lee moon soo <mo...@apache.org>
Committed: Wed May 13 01:49:17 2015 +0200

----------------------------------------------------------------------
 spark/src/main/resources/python/zeppelin_pyspark.py | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-zeppelin/blob/aaee7197/spark/src/main/resources/python/zeppelin_pyspark.py
----------------------------------------------------------------------
diff --git a/spark/src/main/resources/python/zeppelin_pyspark.py b/spark/src/main/resources/python/zeppelin_pyspark.py
index 2e99c44..d5b8daa 100644
--- a/spark/src/main/resources/python/zeppelin_pyspark.py
+++ b/spark/src/main/resources/python/zeppelin_pyspark.py
@@ -15,9 +15,10 @@
 # limitations under the License.
 #
 
-import sys, getopt
+import sys, getopt, traceback
 
 from py4j.java_gateway import java_import, JavaGateway, GatewayClient
+from py4j.protocol import Py4JJavaError
 from pyspark.conf import SparkConf
 from pyspark.context import SparkContext
 from pyspark.rdd import RDD
@@ -106,6 +107,12 @@ while True :
       eval(compiledCode)
 
     intp.setStatementsFinished(output.get(), False)
+  except Py4JJavaError:
+    excInnerError = traceback.format_exc() # format_tb() does not return the inner exception
+    innerErrorStart = excInnerError.find("Py4JJavaError:")
+    if innerErrorStart > -1:
+       excInnerError = excInnerError[innerErrorStart:]
+    intp.setStatementsFinished(excInnerError + str(sys.exc_info()), True)
   except:
     intp.setStatementsFinished(str(sys.exc_info()), True)