Skip to content

Commit deac481

Browse files
itholicHyukjinKwon
authored andcommitted
[SPARK-42824][CONNECT][PYTHON] Provide a clear error message for unsupported JVM attributes
### What changes were proposed in this pull request? This pull request proposes an improvement to the error message when trying to access a JVM attribute that is not supported in Spark Connect. Specifically, it adds a more informative error message that clearly indicates which attribute is not supported due to Spark Connect's lack of dependency on the JVM. ### Why are the changes needed? Currently, when attempting to access an unsupported JVM attribute in Spark Connect, the error message is not very clear, making it difficult for users to understand the root cause of the issue. This improvement aims to provide more helpful information to users to address this problem as below: **Before** ```python >>> spark._jsc Traceback (most recent call last): File "<stdin>", line 1, in <module> AttributeError: 'SparkSession' object has no attribute '_jsc' ``` **After** ```python >>> spark._jsc Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/Users/haejoon.lee/Desktop/git_store/spark/python/pyspark/sql/connect/session.py", line 490, in _jsc raise PySparkAttributeError( pyspark.errors.exceptions.base.PySparkAttributeError: [JVM_ATTRIBUTE_NOT_SUPPORTED] Attribute `_jsc` is not supported in Spark Connect as it depends on the JVM. If you need to use this attribute, use the original PySpark instead of Spark Connect. ``` ### Does this PR introduce _any_ user-facing change? This PR does not introduce any user-facing change in terms of functionality. However, it improves the error message, which could potentially affect the user experience in a positive way. ### How was this patch tested? This patch was tested by adding new unit tests that specifically target the error message related to unsupported JVM attributes. The tests were run locally on a development environment. Closes apache#40458 from itholic/SPARK-42824. Authored-by: itholic <haejoon.lee@databricks.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
1 parent 1b40565 commit deac481

File tree

8 files changed

+110
-3
lines changed

8 files changed

+110
-3
lines changed

python/pyspark/errors/__init__.py

+2
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
SparkUpgradeException,
3232
PySparkTypeError,
3333
PySparkValueError,
34+
PySparkAttributeError,
3435
)
3536

3637

@@ -47,4 +48,5 @@
4748
"SparkUpgradeException",
4849
"PySparkTypeError",
4950
"PySparkValueError",
51+
"PySparkAttributeError",
5052
]

python/pyspark/errors/error_classes.py

+5
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,11 @@
3939
"Function `<func_name>` should return Column, got <return_type>."
4040
]
4141
},
42+
"JVM_ATTRIBUTE_NOT_SUPPORTED" : {
43+
"message" : [
44+
"Attribute `<attr_name>` is not supported in Spark Connect as it depends on the JVM. If you need to use this attribute, do not use Spark Connect when creating your session."
45+
]
46+
},
4247
"NOT_BOOL" : {
4348
"message" : [
4449
"Argument `<arg_name>` should be a bool, got <arg_type>."

python/pyspark/errors/exceptions/base.py

+6
Original file line numberDiff line numberDiff line change
@@ -160,3 +160,9 @@ class PySparkTypeError(PySparkException, TypeError):
160160
"""
161161
Wrapper class for TypeError to support error classes.
162162
"""
163+
164+
165+
class PySparkAttributeError(PySparkException, AttributeError):
166+
"""
167+
Wrapper class for AttributeError to support error classes.
168+
"""

python/pyspark/sql/connect/column.py

+11-1
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@
3131
Optional,
3232
)
3333

34-
from pyspark.errors import PySparkTypeError
34+
from pyspark.errors import PySparkTypeError, PySparkAttributeError
3535
from pyspark.sql.types import DataType
3636
from pyspark.sql.column import Column as PySparkColumn
3737

@@ -433,6 +433,10 @@ def dropFields(self, *fieldNames: str) -> "Column":
433433
dropFields.__doc__ = PySparkColumn.dropFields.__doc__
434434

435435
def __getattr__(self, item: Any) -> "Column":
436+
if item == "_jc":
437+
raise PySparkAttributeError(
438+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED", message_parameters={"attr_name": "_jc"}
439+
)
436440
if item.startswith("__"):
437441
raise AttributeError(item)
438442
return self[item]
@@ -459,6 +463,12 @@ def __nonzero__(self) -> None:
459463

460464
__bool__ = __nonzero__
461465

466+
@property
467+
def _jc(self) -> None:
468+
raise PySparkAttributeError(
469+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED", message_parameters={"attr_name": "_jc"}
470+
)
471+
462472

463473
Column.__doc__ = PySparkColumn.__doc__
464474

python/pyspark/sql/connect/dataframe.py

+5-1
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@
5050
DataFrameStatFunctions as PySparkDataFrameStatFunctions,
5151
)
5252

53-
from pyspark.errors import PySparkTypeError
53+
from pyspark.errors import PySparkTypeError, PySparkAttributeError
5454
from pyspark.errors.exceptions.connect import SparkConnectException
5555
from pyspark.rdd import PythonEvalType
5656
import pyspark.sql.connect.plan as plan
@@ -1304,6 +1304,10 @@ def _get_alias(self) -> Optional[str]:
13041304
return None
13051305

13061306
def __getattr__(self, name: str) -> "Column":
1307+
if name in ["_jseq", "_jdf", "_jmap", "_jcols"]:
1308+
raise PySparkAttributeError(
1309+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED", message_parameters={"attr_name": name}
1310+
)
13071311
return self[name]
13081312

13091313
@overload

python/pyspark/sql/connect/readwriter.py

+7
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030
DataFrameReader as PySparkDataFrameReader,
3131
DataFrameWriterV2 as PySparkDataFrameWriterV2,
3232
)
33+
from pyspark.errors import PySparkAttributeError
3334

3435
if TYPE_CHECKING:
3536
from pyspark.sql.connect.dataframe import DataFrame
@@ -417,6 +418,12 @@ def jdbc(
417418

418419
jdbc.__doc__ = PySparkDataFrameReader.jdbc.__doc__
419420

421+
@property
422+
def _jreader(self) -> None:
423+
raise PySparkAttributeError(
424+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED", message_parameters={"attr_name": "_jreader"}
425+
)
426+
420427

421428
DataFrameReader.__doc__ = PySparkDataFrameReader.__doc__
422429

python/pyspark/sql/connect/session.py

+26
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,7 @@
6767
TimestampType,
6868
)
6969
from pyspark.sql.utils import to_str
70+
from pyspark.errors import PySparkAttributeError
7071

7172
if TYPE_CHECKING:
7273
from pyspark.sql.connect._typing import OptionalPrimitiveType
@@ -484,6 +485,31 @@ def streams(self) -> Any:
484485
def readStream(self) -> Any:
485486
raise NotImplementedError("readStream() is not implemented.")
486487

488+
@property
489+
def _jsc(self) -> None:
490+
raise PySparkAttributeError(
491+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED", message_parameters={"attr_name": "_jsc"}
492+
)
493+
494+
@property
495+
def _jconf(self) -> None:
496+
raise PySparkAttributeError(
497+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED", message_parameters={"attr_name": "_jconf"}
498+
)
499+
500+
@property
501+
def _jvm(self) -> None:
502+
raise PySparkAttributeError(
503+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED", message_parameters={"attr_name": "_jvm"}
504+
)
505+
506+
@property
507+
def _jsparkSession(self) -> None:
508+
raise PySparkAttributeError(
509+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED",
510+
message_parameters={"attr_name": "_jsparkSession"},
511+
)
512+
487513
@property
488514
def udf(self) -> "UDFRegistration":
489515
from pyspark.sql.connect.udf import UDFRegistration

python/pyspark/sql/tests/connect/test_connect_basic.py

+48-1
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,7 @@
2323
import tempfile
2424
from collections import defaultdict
2525

26-
from pyspark.errors import PySparkTypeError
26+
from pyspark.errors import PySparkAttributeError, PySparkTypeError
2727
from pyspark.sql import SparkSession as PySparkSession, Row
2828
from pyspark.sql.types import (
2929
StructType,
@@ -2936,6 +2936,53 @@ def test_map_has_nullable(self):
29362936
self.assertEqual(cdf2.schema, sdf2.schema)
29372937
self.assertEqual(cdf2.collect(), sdf2.collect())
29382938

2939+
def test_unsupported_jvm_attribute(self):
2940+
# Unsupported jvm attributes for Spark session.
2941+
unsupported_attrs = ["_jsc", "_jconf", "_jvm", "_jsparkSession"]
2942+
spark_session = self.connect
2943+
for attr in unsupported_attrs:
2944+
with self.assertRaises(PySparkAttributeError) as pe:
2945+
getattr(spark_session, attr)
2946+
2947+
self.check_error(
2948+
exception=pe.exception,
2949+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED",
2950+
message_parameters={"attr_name": attr},
2951+
)
2952+
2953+
# Unsupported jvm attributes for DataFrame.
2954+
unsupported_attrs = ["_jseq", "_jdf", "_jmap", "_jcols"]
2955+
cdf = self.connect.range(10)
2956+
for attr in unsupported_attrs:
2957+
with self.assertRaises(PySparkAttributeError) as pe:
2958+
getattr(cdf, attr)
2959+
2960+
self.check_error(
2961+
exception=pe.exception,
2962+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED",
2963+
message_parameters={"attr_name": attr},
2964+
)
2965+
2966+
# Unsupported jvm attributes for Column.
2967+
with self.assertRaises(PySparkAttributeError) as pe:
2968+
getattr(cdf.id, "_jc")
2969+
2970+
self.check_error(
2971+
exception=pe.exception,
2972+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED",
2973+
message_parameters={"attr_name": "_jc"},
2974+
)
2975+
2976+
# Unsupported jvm attributes for DataFrameReader.
2977+
with self.assertRaises(PySparkAttributeError) as pe:
2978+
getattr(spark_session.read, "_jreader")
2979+
2980+
self.check_error(
2981+
exception=pe.exception,
2982+
error_class="JVM_ATTRIBUTE_NOT_SUPPORTED",
2983+
message_parameters={"attr_name": "_jreader"},
2984+
)
2985+
29392986

29402987
@unittest.skipIf(not should_test_connect, connect_requirement_message)
29412988
class ClientTests(unittest.TestCase):

0 commit comments

Comments
 (0)