You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by sr...@apache.org on 2023/03/14 13:30:51 UTC

[spark] branch master updated: [SPARK-42752][PYSPARK][SQL] Make PySpark exceptions printable during initialization

This is an automated email from the ASF dual-hosted git repository.

srowen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new b2a7f14cbd8 [SPARK-42752][PYSPARK][SQL] Make PySpark exceptions printable during initialization
b2a7f14cbd8 is described below

commit b2a7f14cbd8fd3b1a51d7b53fc7c23fb71e9f370
Author: Gera Shegalov <ge...@apache.org>
AuthorDate: Tue Mar 14 08:30:15 2023 -0500

    [SPARK-42752][PYSPARK][SQL] Make PySpark exceptions printable during initialization
    
    Ignore SQLConf initialization exceptions during Python exception creation.
    
    Otherwise there is no diagnostics for the issue in the following scenario:
    
    1. download a standard "Hadoop Free" build
    2. Start PySpark REPL with Hive support
    ```bash
    SPARK_DIST_CLASSPATH=$(~/dist/hadoop-3.4.0-SNAPSHOT/bin/hadoop classpath) \
      ~/dist/spark-3.2.3-bin-without-hadoop/bin/pyspark --conf spark.sql.catalogImplementation=hive
    ```
    3. Execute any simple dataframe operation
    ```Python
    >>> spark.range(100).show()
    Traceback (most recent call last):
      File "<stdin>", line 1, in <module>
      File "/home/user/dist/spark-3.2.3-bin-without-hadoop/python/pyspark/sql/session.py", line 416, in range
        jdf = self._jsparkSession.range(0, int(start), int(step), int(numPartitions))
      File "/home/user/dist/spark-3.2.3-bin-without-hadoop/python/lib/py4j-0.10.9.5-src.zip/py4j/java_gateway.py", line 1321, in __call__
      File "/home/user/dist/spark-3.2.3-bin-without-hadoop/python/pyspark/sql/utils.py", line 117, in deco
        raise converted from None
    pyspark.sql.utils.IllegalArgumentException: <exception str() failed>
    ```
    4. In fact just spark.conf already exhibits the issue
    ```Python
    >>> spark.conf
    Traceback (most recent call last):
      File "<stdin>", line 1, in <module>
      File "/home/user/dist/spark-3.2.3-bin-without-hadoop/python/pyspark/sql/session.py", line 347, in conf
        self._conf = RuntimeConfig(self._jsparkSession.conf())
      File "/home/user/dist/spark-3.2.3-bin-without-hadoop/python/lib/py4j-0.10.9.5-src.zip/py4j/java_gateway.py", line 1321, in __call__
      File "/home/user/dist/spark-3.2.3-bin-without-hadoop/python/pyspark/sql/utils.py", line 117, in deco
        raise converted from None
    pyspark.sql.utils.IllegalArgumentException: <exception str() failed>
    ```
    
    There are probably two issues here:
    1) that Hive support should be gracefully disabled if it the dependency not on the classpath as claimed by https://spark.apache.org/docs/latest/sql-data-sources-hive-tables.html
    2) but at the very least the user should be able to see the exception to understand the issue, and take an action
    
    ### What changes were proposed in this pull request?
    
    Ignore exceptions during `CapturedException` creation
    
    ### Why are the changes needed?
    To make the cause visible to the user
    
    ```Python
    Traceback (most recent call last):
      File "<stdin>", line 1, in <module>
      File "/home/user/gits/apache/spark/python/pyspark/sql/session.py", line 679, in conf
        self._conf = RuntimeConfig(self._jsparkSession.conf())
      File "/home/user/gits/apache/spark/python/lib/py4j-0.10.9.7-src.zip/py4j/java_gateway.py", line 1322, in __call__
      File "/home/user/gits/apache/spark/python/pyspark/errors/exceptions/captured.py", line 166, in deco
        raise converted from None
    pyspark.errors.exceptions.captured.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':
    
    JVM stacktrace:
    java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':
            at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1237)
            at org.apache.spark.sql.SparkSession.$anonfun$sessionState$2(SparkSession.scala:162)
            at scala.Option.getOrElse(Option.scala:189)
            at org.apache.spark.sql.SparkSession.sessionState$lzycompute(SparkSession.scala:160)
            at org.apache.spark.sql.SparkSession.sessionState(SparkSession.scala:157)
            at org.apache.spark.sql.SparkSession.conf$lzycompute(SparkSession.scala:185)
            at org.apache.spark.sql.SparkSession.conf(SparkSession.scala:185)
            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:374)
            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.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
            at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
            at java.lang.Thread.run(Thread.java:750)
    Caused by: java.lang.ClassNotFoundException: org.apache.spark.sql.hive.HiveSessionStateBuilder
            at java.net.URLClassLoader.findClass(URLClassLoader.java:387)
            at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
            at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
            at java.lang.Class.forName0(Native Method)
            at java.lang.Class.forName(Class.java:348)
            at org.apache.spark.util.Utils$.classForName(Utils.scala:225)
            at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1232)
            ... 18 more
    ```
    
    ### Does this PR introduce _any_ user-facing change?
    The only semantic change is that the conf `spark.sql.pyspark.jvmStacktrace.enabled` is ignored if the SQLConf is broken.
    
    ### How was this patch tested?
    Manual testing using the repro steps above
    
    Closes #40372 from gerashegalov/SPARK-42752.
    
    Authored-by: Gera Shegalov <ge...@apache.org>
    Signed-off-by: Sean Owen <sr...@gmail.com>
---
 python/pyspark/errors/exceptions/captured.py | 11 +++++++++--
 1 file changed, 9 insertions(+), 2 deletions(-)

diff --git a/python/pyspark/errors/exceptions/captured.py b/python/pyspark/errors/exceptions/captured.py
index 1764ed7d02c..6313665b3fe 100644
--- a/python/pyspark/errors/exceptions/captured.py
+++ b/python/pyspark/errors/exceptions/captured.py
@@ -65,8 +65,15 @@ class CapturedException(PySparkException):
         assert SparkContext._jvm is not None
 
         jvm = SparkContext._jvm
-        sql_conf = jvm.org.apache.spark.sql.internal.SQLConf.get()
-        debug_enabled = sql_conf.pysparkJVMStacktraceEnabled()
+
+        # SPARK-42752: default to True to see issues with initialization
+        debug_enabled = True
+        try:
+            sql_conf = jvm.org.apache.spark.sql.internal.SQLConf.get()
+            debug_enabled = sql_conf.pysparkJVMStacktraceEnabled()
+        except BaseException:
+            pass
+
         desc = self.desc
         if debug_enabled:
             desc = desc + "\n\nJVM stacktrace:\n%s" % self.stackTrace


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