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 1f8422b77ae [SPARK-45674][CONNECT][PYTHON] Improve error message for JVM-dependent attributes on Spark Connect 1f8422b77ae is described below commit 1f8422b77ae9fd920a30c55947712a5fb388c480 Author: Haejoon Lee <haejoon....@databricks.com> AuthorDate: Sun Oct 29 19:26:50 2023 +0900 [SPARK-45674][CONNECT][PYTHON] Improve error message for JVM-dependent attributes on Spark Connect ### What changes were proposed in this pull request? This PR proposes to improve error message for JVM-dependent attributes on Spark Connect. ### Why are the changes needed? To improve the usability of error message by using proper error class. ### Does this PR introduce _any_ user-facing change? No API change, but only user-facing error messages are improved as below: **Before** ``` >>> spark.sparkContext PySparkNotImplementedError: [NOT_IMPLEMENTED] sparkContext() is not implemented. ``` **After** ``` >>> spark.sparkContext PySparkAttributeError: [JVM_ATTRIBUTE_NOT_SUPPORTED] Attribute `sparkContext` 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. ``` ### How was this patch tested? The existing CI should pass ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43537 from itholic/SPARK-45674. Authored-by: Haejoon Lee <haejoon....@databricks.com> Signed-off-by: Hyukjin Kwon <gurwls...@apache.org> --- python/pyspark/errors/error_classes.py | 2 +- python/pyspark/sql/connect/dataframe.py | 4 +--- python/pyspark/sql/connect/session.py | 6 +----- python/pyspark/sql/tests/connect/test_connect_basic.py | 14 -------------- 4 files changed, 3 insertions(+), 23 deletions(-) diff --git a/python/pyspark/errors/error_classes.py b/python/pyspark/errors/error_classes.py index 53a7279fde2..70e88c18f9d 100644 --- a/python/pyspark/errors/error_classes.py +++ b/python/pyspark/errors/error_classes.py @@ -354,7 +354,7 @@ ERROR_CLASSES_JSON = """ }, "JVM_ATTRIBUTE_NOT_SUPPORTED" : { "message" : [ - "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." + "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. Visit https://spark.apache.org/docs/latest/sql-getting-started.html#starting-point-sparksession for creating regular Spark Session in detail." ] }, "KEY_VALUE_PAIR_REQUIRED" : { diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index b322ded84a4..75cecd5f610 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1639,13 +1639,11 @@ class DataFrame: sampleBy.__doc__ = PySparkDataFrame.sampleBy.__doc__ def __getattr__(self, name: str) -> "Column": - if name in ["_jseq", "_jdf", "_jmap", "_jcols"]: + if name in ["_jseq", "_jdf", "_jmap", "_jcols", "rdd", "toJSON"]: raise PySparkAttributeError( error_class="JVM_ATTRIBUTE_NOT_SUPPORTED", message_parameters={"attr_name": name} ) elif name in [ - "rdd", - "toJSON", "checkpoint", "localCheckpoint", ]: diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 53bf19b78c8..09bd60606c7 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -694,14 +694,10 @@ class SparkSession: streams.__doc__ = PySparkSession.streams.__doc__ def __getattr__(self, name: str) -> Any: - if name in ["_jsc", "_jconf", "_jvm", "_jsparkSession"]: + if name in ["_jsc", "_jconf", "_jvm", "_jsparkSession", "sparkContext", "newSession"]: raise PySparkAttributeError( error_class="JVM_ATTRIBUTE_NOT_SUPPORTED", message_parameters={"attr_name": name} ) - elif name in ["newSession", "sparkContext"]: - raise PySparkNotImplementedError( - error_class="NOT_IMPLEMENTED", message_parameters={"feature": f"{name}()"} - ) return object.__getattribute__(self, name) @property diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index c96d08b5bbe..34bd314c76f 100755 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -3097,26 +3097,12 @@ class SparkConnectBasicTests(SparkConnectSQLTestCase): # SPARK-41225: Disable unsupported functions. df = self.connect.read.table(self.tbl_name) for f in ( - "rdd", "checkpoint", "localCheckpoint", ): with self.assertRaises(NotImplementedError): getattr(df, f)() - def test_unsupported_session_functions(self): - # SPARK-41934: Disable unsupported functions. - - with self.assertRaises(NotImplementedError): - RemoteSparkSession.builder.enableHiveSupport() - - for f in ( - "newSession", - "sparkContext", - ): - with self.assertRaises(NotImplementedError): - getattr(self.connect, f)() - def test_sql_with_command(self): # SPARK-42705: spark.sql should return values from the command. self.assertEqual( --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org