Skip to content

Commit d2a1a31

Browse files
allisonwang-dbHyukjinKwon
authored andcommitted
[SPARK-45783][PYTHON][CONNECT] Improve error messages when Spark Connect mode is enabled but remote URL is not set
### What changes were proposed in this pull request? This PR improves the error messages when `SPARK_CONNECT_MODE_ENABLED` is defined but neither `spark.remote` option nor the `SPARK_REMOTE` env var is set. ### Why are the changes needed? To improve the error message. Currently the error looks like a bug: ``` url = opts.get("spark.remote", os.environ.get("SPARK_REMOTE")) > if url.startswith("local"): E AttributeError: 'NoneType' object has no attribute 'startswith' ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #43653 from allisonwang-db/spark-45783-fix-url-err. Authored-by: allisonwang-db <[email protected]> Signed-off-by: Hyukjin Kwon <[email protected]>
1 parent c29b127 commit d2a1a31

File tree

2 files changed

+14
-0
lines changed

2 files changed

+14
-0
lines changed

python/pyspark/sql/session.py

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -481,6 +481,14 @@ def getOrCreate(self) -> "SparkSession":
481481
):
482482
url = opts.get("spark.remote", os.environ.get("SPARK_REMOTE"))
483483

484+
if url is None:
485+
raise RuntimeError(
486+
"Cannot create a Spark Connect session because the "
487+
"Spark Connect remote URL has not been set. Please define "
488+
"the remote URL by setting either the 'spark.remote' option "
489+
"or the 'SPARK_REMOTE' environment variable."
490+
)
491+
484492
if url.startswith("local"):
485493
os.environ["SPARK_LOCAL_REMOTE"] = "1"
486494
RemoteSparkSession._start_connect_server(url, opts)

python/pyspark/sql/tests/test_session.py

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
import os
1919
import unittest
20+
import unittest.mock
2021

2122
from pyspark import SparkConf, SparkContext
2223
from pyspark.sql import SparkSession, SQLContext, Row
@@ -187,6 +188,11 @@ def test_active_session_with_None_and_not_None_context(self):
187188
if sc is not None:
188189
sc.stop()
189190

191+
def test_session_with_spark_connect_mode_enabled(self):
192+
with unittest.mock.patch.dict(os.environ, {"SPARK_CONNECT_MODE_ENABLED": "1"}):
193+
with self.assertRaisesRegex(RuntimeError, "Cannot create a Spark Connect session"):
194+
SparkSession.builder.appName("test").getOrCreate()
195+
190196

191197
class SparkSessionTests4(ReusedSQLTestCase):
192198
def test_get_active_session_after_create_dataframe(self):

0 commit comments

Comments
 (0)