HyukjinKwon opened a new pull request #28661:
URL: https://github.com/apache/spark/pull/28661


   ### What changes were proposed in this pull request?
   
   This PR proposes to make PySpark exception more Pythonic by hiding JVM 
stacktrace by default. It can be enabled by turning on 
`spark.sql.pyspark.jvmStackTrace.enabled` configuration.
   
   ```
   Traceback (most recent call last):
     ...
   pyspark.sql.utils.PythonException:
     An exception was thrown from Python worker in the executor. The below is 
the Python worker stacktrace.
   Traceback (most recent call last):
     ...
   ```
   
   If this `spark.sql.pyspark.jvmStackTrace.enabled` is enabled, it appends:
   
   ```
   JVM stacktrace:
   org.apache.spark.Exception: ...
     ...
   ```
   
   For example, the codes below:
   
   ```python
   from pyspark.sql.functions import udf
   @udf
   def divide_by_zero(v):
       raise v / 0
   
   spark.range(1).select(divide_by_zero("id")).show()
   ```
   
   will show an error messages that looks like Python exception thrown from the 
local. 
   
   <details>
   <summary>Python exception message when 
<code>spark.sql.pyspark.jvmStackTrace.enabled</code> is off (default)</summary>
   
   ```
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "/.../spark/python/pyspark/sql/dataframe.py", line 427, in show
       print(self._jdf.showString(n, 20, vertical))
     File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", 
line 1305, in __call__
     File "/.../spark/python/pyspark/sql/utils.py", line 131, in deco
       raise_from(converted)
     File "<string>", line 3, in raise_from
   pyspark.sql.utils.PythonException:
     An exception was thrown from Python worker in the executor. The below is 
the Python worker stacktrace.
   Traceback (most recent call last):
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in 
main
       process()
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in 
process
       serializer.dump_stream(out_iter, outfile)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, 
in dump_stream
       self.serializer.dump_stream(self._batched(iterator), stream)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, 
in dump_stream
       for obj in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, 
in _batched
       for item in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
mapper
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
<genexpr>
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in 
<lambda>
       return lambda *a: f(*a)
     File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in 
wrapper
       return f(*args, **kwargs)
     File "<stdin>", line 3, in divide_by_zero
   ZeroDivisionError: division by zero
   ```
   
   </details>
   
   <details>
   <summary>Python exception message when 
<code>spark.sql.pyspark.jvmStackTrace.enabled</code> is on</summary>
   
   ```
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "/.../spark/python/pyspark/sql/dataframe.py", line 427, in show
       print(self._jdf.showString(n, 20, vertical))
     File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", 
line 1305, in __call__
     File "/.../spark/python/pyspark/sql/utils.py", line 137, in deco
       raise_from(converted)
     File "<string>", line 3, in raise_from
   pyspark.sql.utils.PythonException:
     An exception was thrown from Python worker in the executor. The below is 
the Python worker stacktrace.
   Traceback (most recent call last):
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in 
main
       process()
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in 
process
       serializer.dump_stream(out_iter, outfile)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, 
in dump_stream
       self.serializer.dump_stream(self._batched(iterator), stream)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, 
in dump_stream
       for obj in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, 
in _batched
       for item in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
mapper
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
<genexpr>
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in 
<lambda>
       return lambda *a: f(*a)
     File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in 
wrapper
       return f(*args, **kwargs)
     File "<stdin>", line 3, in divide_by_zero
   ZeroDivisionError: division by zero
   
   JVM stacktrace:
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in 
stage 1.0 failed 4 times, most recent failure: Lost task 0.3 in stage 1.0 (TID 
4, 192.168.35.193, executor 0): org.apache.spark.api.python.PythonException: 
Traceback (most recent call last):
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in 
main
       process()
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in 
process
       serializer.dump_stream(out_iter, outfile)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, 
in dump_stream
       self.serializer.dump_stream(self._batched(iterator), stream)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, 
in dump_stream
       for obj in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, 
in _batched
       for item in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
mapper
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
<genexpr>
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in 
<lambda>
       return lambda *a: f(*a)
     File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in 
wrapper
       return f(*args, **kwargs)
     File "<stdin>", line 3, in divide_by_zero
   ZeroDivisionError: division by zero
   
        at 
org.apache.spark.api.python.BasePythonRunner$ReaderIterator.handlePythonException(PythonRunner.scala:516)
        at 
org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:81)
        at 
org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:64)
        at 
org.apache.spark.api.python.BasePythonRunner$ReaderIterator.hasNext(PythonRunner.scala:469)
        at 
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:127)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:469)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:472)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
   
   Driver stacktrace:
        at 
org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2117)
        at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2066)
        at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2065)
        at 
scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
        at 
scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
        at 
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2065)
        at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1021)
        at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1021)
        at scala.Option.foreach(Option.scala:407)
        at 
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1021)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2297)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2246)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2235)
        at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
        at 
org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:823)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2108)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2129)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2148)
        at 
org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:467)
        at 
org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:420)
        at 
org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:47)
        at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3653)
        at org.apache.spark.sql.Dataset.$anonfun$head$1(Dataset.scala:2695)
        at 
org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
        at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
        at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
        at org.apache.spark.sql.Dataset.head(Dataset.scala:2695)
        at org.apache.spark.sql.Dataset.take(Dataset.scala:2902)
        at org.apache.spark.sql.Dataset.getRows(Dataset.scala:300)
        at org.apache.spark.sql.Dataset.showString(Dataset.scala:337)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
        at py4j.Gateway.invoke(Gateway.java:282)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at py4j.GatewayConnection.run(GatewayConnection.java:238)
        at java.lang.Thread.run(Thread.java:748)
   Caused by: org.apache.spark.api.python.PythonException: Traceback (most 
recent call last):
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in 
main
       process()
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in 
process
       serializer.dump_stream(out_iter, outfile)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, 
in dump_stream
       self.serializer.dump_stream(self._batched(iterator), stream)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, 
in dump_stream
       for obj in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, 
in _batched
       for item in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
mapper
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
<genexpr>
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in 
<lambda>
       return lambda *a: f(*a)
     File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in 
wrapper
       return f(*args, **kwargs)
     File "<stdin>", line 3, in divide_by_zero
   ZeroDivisionError: division by zero
   
        at 
org.apache.spark.api.python.BasePythonRunner$ReaderIterator.handlePythonException(PythonRunner.scala:516)
        at 
org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:81)
        at 
org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:64)
        at 
org.apache.spark.api.python.BasePythonRunner$ReaderIterator.hasNext(PythonRunner.scala:469)
        at 
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:127)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:469)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:472)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        ... 1 more
   ```
   
   </details>
   
   <details>
   <summary>Python exception message without this change</summary>
   
   ```
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "/.../spark/python/pyspark/sql/dataframe.py", line 427, in show
       print(self._jdf.showString(n, 20, vertical))
     File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", 
line 1305, in __call__
     File "/.../spark/python/pyspark/sql/utils.py", line 98, in deco
       return f(*a, **kw)
     File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/protocol.py", line 
328, in get_return_value
   py4j.protocol.Py4JJavaError: An error occurred while calling o160.showString.
   : org.apache.spark.SparkException: Job aborted due to stage failure: Task 10 
in stage 5.0 failed 4 times, most recent failure: Lost task 10.3 in stage 5.0 
(TID 37, 192.168.35.193, executor 3): 
org.apache.spark.api.python.PythonException: Traceback (most recent call last):
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in 
main
       process()
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in 
process
       serializer.dump_stream(out_iter, outfile)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, 
in dump_stream
       self.serializer.dump_stream(self._batched(iterator), stream)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, 
in dump_stream
       for obj in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, 
in _batched
       for item in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
mapper
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
<genexpr>
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in 
<lambda>
       return lambda *a: f(*a)
     File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in 
wrapper
       return f(*args, **kwargs)
     File "<stdin>", line 3, in divide_by_zero
   ZeroDivisionError: division by zero
   
        at 
org.apache.spark.api.python.BasePythonRunner$ReaderIterator.handlePythonException(PythonRunner.scala:516)
        at 
org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:81)
        at 
org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:64)
        at 
org.apache.spark.api.python.BasePythonRunner$ReaderIterator.hasNext(PythonRunner.scala:469)
        at 
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:127)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:469)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:472)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
   
   Driver stacktrace:
        at 
org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2117)
        at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2066)
        at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2065)
        at 
scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
        at 
scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
        at 
org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2065)
        at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1021)
        at 
org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1021)
        at scala.Option.foreach(Option.scala:407)
        at 
org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1021)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2297)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2246)
        at 
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2235)
        at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
        at 
org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:823)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2108)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2129)
        at org.apache.spark.SparkContext.runJob(SparkContext.scala:2148)
        at 
org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:467)
        at 
org.apache.spark.sql.execution.SparkPlan.executeTake(SparkPlan.scala:420)
        at 
org.apache.spark.sql.execution.CollectLimitExec.executeCollect(limit.scala:47)
        at org.apache.spark.sql.Dataset.collectFromPlan(Dataset.scala:3653)
        at org.apache.spark.sql.Dataset.$anonfun$head$1(Dataset.scala:2695)
        at 
org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
        at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
        at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
        at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
        at org.apache.spark.sql.Dataset.head(Dataset.scala:2695)
        at org.apache.spark.sql.Dataset.take(Dataset.scala:2902)
        at org.apache.spark.sql.Dataset.getRows(Dataset.scala:300)
        at org.apache.spark.sql.Dataset.showString(Dataset.scala:337)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
        at py4j.Gateway.invoke(Gateway.java:282)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at py4j.GatewayConnection.run(GatewayConnection.java:238)
        at java.lang.Thread.run(Thread.java:748)
   Caused by: org.apache.spark.api.python.PythonException: Traceback (most 
recent call last):
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 605, in 
main
       process()
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 597, in 
process
       serializer.dump_stream(out_iter, outfile)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 223, 
in dump_stream
       self.serializer.dump_stream(self._batched(iterator), stream)
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 141, 
in dump_stream
       for obj in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/serializers.py", line 212, 
in _batched
       for item in iterator:
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
mapper
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 450, in 
<genexpr>
       result = tuple(f(*[a[o] for o in arg_offsets]) for (arg_offsets, f) in 
udfs)
     File "/.../spark/python/lib/pyspark.zip/pyspark/worker.py", line 90, in 
<lambda>
       return lambda *a: f(*a)
     File "/.../spark/python/lib/pyspark.zip/pyspark/util.py", line 107, in 
wrapper
       return f(*args, **kwargs)
     File "<stdin>", line 3, in divide_by_zero
   ZeroDivisionError: division by zero
   
        at 
org.apache.spark.api.python.BasePythonRunner$ReaderIterator.handlePythonException(PythonRunner.scala:516)
        at 
org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:81)
        at 
org.apache.spark.sql.execution.python.PythonUDFRunner$$anon$2.read(PythonUDFRunner.scala:64)
        at 
org.apache.spark.api.python.BasePythonRunner$ReaderIterator.hasNext(PythonRunner.scala:469)
        at 
org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:37)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:489)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:458)
        at 
org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage2.processNext(Unknown
 Source)
        at 
org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
        at 
org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:753)
        at 
org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:340)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
        at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
        at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:127)
        at 
org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:469)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:472)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        ... 1 more
   ```
   
   
   </details>
   
   <br/>
   
   Another example with Python 3.7:
   
   ```python
   sql("a")
   ```
   
   <details>
   <summary>Python exception message when 
<code>spark.sql.pyspark.jvmStackTrace.enabled</code> is off (default)</summary>
   
   ```
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "/.../spark/python/pyspark/sql/session.py", line 646, in sql
       return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
     File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", 
line 1305, in __call__
     File "/.../spark/python/pyspark/sql/utils.py", line 131, in deco
       raise_from(converted)
     File "<string>", line 3, in raise_from
   pyspark.sql.utils.ParseException:
   mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 
'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 
'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 
'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 
'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 
'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 
0)
   
   == SQL ==
   a
   ^^^
   ```
   
   </details>
   
   <details>
   <summary>Python exception message when 
<code>spark.sql.pyspark.jvmStackTrace.enabled</code> is on (default)</summary>
   
   ```
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "/.../spark/python/pyspark/sql/session.py", line 646, in sql
       return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
     File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", 
line 1305, in __call__
     File "/.../spark/python/pyspark/sql/utils.py", line 131, in deco
       raise_from(converted)
     File "<string>", line 3, in raise_from
   pyspark.sql.utils.ParseException:
   mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 
'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 
'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 
'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 
'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 
'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 
0)
   
   == SQL ==
   a
   ^^^
   
   JVM stacktrace:
   org.apache.spark.sql.catalyst.parser.ParseException:
   mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 
'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 
'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 
'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 
'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 
'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 
0)
   
   == SQL ==
   a
   ^^^
   
        at 
org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:266)
        at 
org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:133)
        at 
org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:49)
        at 
org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
        at 
org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:604)
        at 
org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
        at 
org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:604)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
        at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:601)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
        at py4j.Gateway.invoke(Gateway.java:282)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at py4j.GatewayConnection.run(GatewayConnection.java:238)
        at java.lang.Thread.run(Thread.java:748)
   ```
   
   
   </details>
   
   
   <details>
   <summary>Python exception message without this change</summary>
   
   ```
   Traceback (most recent call last):
     File "/.../spark/python/pyspark/sql/utils.py", line 98, in deco
       return f(*a, **kw)
     File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/protocol.py", line 
328, in get_return_value
   py4j.protocol.Py4JJavaError: An error occurred while calling o26.sql.
   : org.apache.spark.sql.catalyst.parser.ParseException:
   mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 
'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 
'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 
'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 
'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 
'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 
0)
   
   == SQL ==
   a
   ^^^
   
        at 
org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:266)
        at 
org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:133)
        at 
org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:49)
        at 
org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
        at 
org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:604)
        at 
org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
        at 
org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:604)
        at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:763)
        at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:601)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
        at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
        at py4j.Gateway.invoke(Gateway.java:282)
        at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
        at py4j.commands.CallCommand.execute(CallCommand.java:79)
        at py4j.GatewayConnection.run(GatewayConnection.java:238)
        at java.lang.Thread.run(Thread.java:748)
   
   
   During handling of the above exception, another exception occurred:
   
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "/.../spark/python/pyspark/sql/session.py", line 646, in sql
       return DataFrame(self._jsparkSession.sql(sqlQuery), self._wrapped)
     File "/.../spark/python/lib/py4j-0.10.9-src.zip/py4j/java_gateway.py", 
line 1305, in __call__
     File "/.../spark/python/pyspark/sql/utils.py", line 102, in deco
       raise converted
   pyspark.sql.utils.ParseException:
   mismatched input 'a' expecting {'(', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 
'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 
'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 
'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 
'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'TABLE', 
'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 
0)
   
   == SQL ==
   a
   ^^^
   ```
   
   </details>
   
   
   ### Why are the changes needed?
   
   Currently, PySpark exceptions are very unfriendly to Python users with 
causing a bunch of JVM stacktrace. See "Python exception message without this 
change" above.
   
   ### Does this PR introduce _any_ user-facing change?
   
   Yes, it will change the exception message. See the examples above.
   
   ### How was this patch tested?
   
   Manually tested by
   
   ```bash
   ./bin/pyspark --conf spark.sql.pyspark.jvmStackTrace.enabled=true
   ```
   
   and running the examples above..
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to