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

gurwls223 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 4134e9f208bb [SPARK-51178][CONNECT][PYTHON] Raise proper PySpark error 
instead of `SparkConnectGrpcException`
4134e9f208bb is described below

commit 4134e9f208bb53db72f0c7398473c6af63da3838
Author: Haejoon Lee <[email protected]>
AuthorDate: Wed Feb 19 08:35:29 2025 +0900

    [SPARK-51178][CONNECT][PYTHON] Raise proper PySpark error instead of 
`SparkConnectGrpcException`
    
    ### What changes were proposed in this pull request?
    
    This PR propose to raise proper PySpark error instead of 
`SparkConnectGrpcException`.
    
    This PR also introduces new PySpark error `PickleException` to cover the 
errors that represents an exception which is failed while pickling from server 
side
    
    ### Why are the changes needed?
    
    To raise proper exception instead of `SparkConnectGrpcException`
    
    ### Does this PR introduce _any_ user-facing change?
    
    No API changes, but the user-facing error improvement.
    
    ### How was this patch tested?
    
    Updated the existing UT
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No
    
    Closes #49953 from itholic/SPARK-51178.
    
    Authored-by: Haejoon Lee <[email protected]>
    Signed-off-by: Hyukjin Kwon <[email protected]>
---
 python/docs/source/reference/pyspark.errors.rst     |  1 +
 python/pyspark/errors/__init__.py                   |  2 ++
 python/pyspark/errors/exceptions/base.py            |  8 ++++++++
 python/pyspark/errors/exceptions/connect.py         | 21 +++++++++++++++++++++
 .../pyspark/sql/tests/connect/test_parity_udtf.py   |  6 ++----
 5 files changed, 34 insertions(+), 4 deletions(-)

diff --git a/python/docs/source/reference/pyspark.errors.rst 
b/python/docs/source/reference/pyspark.errors.rst
index 60655b342248..4d2bf7612779 100644
--- a/python/docs/source/reference/pyspark.errors.rst
+++ b/python/docs/source/reference/pyspark.errors.rst
@@ -60,6 +60,7 @@ Classes
     TempTableAlreadyExistsException
     UnknownException
     UnsupportedOperationException
+    PickleException
 
 
 Methods
diff --git a/python/pyspark/errors/__init__.py 
b/python/pyspark/errors/__init__.py
index 39bd6698b798..98f8aa593371 100644
--- a/python/pyspark/errors/__init__.py
+++ b/python/pyspark/errors/__init__.py
@@ -51,6 +51,7 @@ from pyspark.errors.exceptions.base import (  # noqa: F401
     QueryContext,
     QueryContextType,
     StreamingPythonRunnerInitializationException,
+    PickleException,
 )
 
 
@@ -87,4 +88,5 @@ __all__ = [
     "QueryContext",
     "QueryContextType",
     "StreamingPythonRunnerInitializationException",
+    "PickleException",
 ]
diff --git a/python/pyspark/errors/exceptions/base.py 
b/python/pyspark/errors/exceptions/base.py
index 21d87e8cd028..13501ba0de78 100644
--- a/python/pyspark/errors/exceptions/base.py
+++ b/python/pyspark/errors/exceptions/base.py
@@ -363,6 +363,14 @@ class PySparkImportError(PySparkException, ImportError):
     """
 
 
+class PickleException(PySparkException):
+    """
+    Represents an exception which is failed while pickling from server side
+    such as `net.razorvine.pickle.PickleException`. This is different from 
`PySparkPicklingError`
+    which represents an exception failed from Python built-in 
`pickle.PicklingError`.
+    """
+
+
 class QueryContextType(Enum):
     """
     The type of :class:`QueryContext`.
diff --git a/python/pyspark/errors/exceptions/connect.py 
b/python/pyspark/errors/exceptions/connect.py
index 5837d89db94d..0da809473a01 100644
--- a/python/pyspark/errors/exceptions/connect.py
+++ b/python/pyspark/errors/exceptions/connect.py
@@ -37,6 +37,7 @@ from pyspark.errors.exceptions.base import (
     QueryContext as BaseQueryContext,
     QueryContextType,
     StreamingPythonRunnerInitializationException as 
BaseStreamingPythonRunnerInitException,
+    PickleException as BasePickleException,
 )
 
 if TYPE_CHECKING:
@@ -94,6 +95,14 @@ def convert_exception(
     # Return exception based on class mapping
     for error_class_name in classes:
         ExceptionClass = EXCEPTION_CLASS_MAPPING.get(error_class_name)
+        if ExceptionClass is SparkException:
+            for third_party_exception_class in 
THIRD_PARTY_EXCEPTION_CLASS_MAPPING:
+                ExceptionClass = (
+                    
THIRD_PARTY_EXCEPTION_CLASS_MAPPING.get(third_party_exception_class)
+                    if third_party_exception_class in message
+                    else SparkException
+                )
+
         if ExceptionClass:
             return ExceptionClass(
                 message,
@@ -316,6 +325,14 @@ class StreamingPythonRunnerInitializationException(
     """
 
 
+class PickleException(SparkConnectGrpcException, BasePickleException):
+    """
+    Represents an exception which is failed while pickling from server side
+    such as `net.razorvine.pickle.PickleException`. This is different from 
`PySparkPicklingError`
+    which represents an exception failed from Python built-in 
`pickle.PicklingError`.
+    """
+
+
 # Update EXCEPTION_CLASS_MAPPING here when adding a new exception
 EXCEPTION_CLASS_MAPPING = {
     "org.apache.spark.sql.catalyst.parser.ParseException": ParseException,
@@ -339,6 +356,10 @@ EXCEPTION_CLASS_MAPPING = {
     "$StreamingPythonRunnerInitializationException": 
StreamingPythonRunnerInitializationException,
 }
 
+THIRD_PARTY_EXCEPTION_CLASS_MAPPING = {
+    "net.razorvine.pickle.PickleException": PickleException,
+}
+
 
 class SQLQueryContext(BaseQueryContext):
     def __init__(self, q: pb2.FetchErrorDetailsResponse.QueryContext):
diff --git a/python/pyspark/sql/tests/connect/test_parity_udtf.py 
b/python/pyspark/sql/tests/connect/test_parity_udtf.py
index 042d31641faf..3d218c312a97 100644
--- a/python/pyspark/sql/tests/connect/test_parity_udtf.py
+++ b/python/pyspark/sql/tests/connect/test_parity_udtf.py
@@ -27,7 +27,7 @@ if should_test_connect:
     sql.udtf.UserDefinedTableFunction = UserDefinedTableFunction
     from pyspark.sql.connect.functions import lit, udtf
     from pyspark.errors.exceptions.connect import (
-        SparkConnectGrpcException,
+        PickleException,
         PythonException,
         InvalidPlanInput,
     )
@@ -46,10 +46,8 @@ class UDTFParityTests(BaseUDTFTestsMixin, 
ReusedConnectTestCase):
         finally:
             super(UDTFParityTests, cls).tearDownClass()
 
-    # TODO: use PySpark error classes instead of SparkConnectGrpcException
-
     def test_struct_output_type_casting_row(self):
-        self.check_struct_output_type_casting_row(SparkConnectGrpcException)
+        self.check_struct_output_type_casting_row(PickleException)
 
     def test_udtf_with_invalid_return_type(self):
         @udtf(returnType="int")


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to