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

dongjoon 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 60a3fbcab4e5 [SPARK-47727][PYTHON] Make SparkConf to root level to for 
both SparkSession and SparkContext
60a3fbcab4e5 is described below

commit 60a3fbcab4e53f89b7128b7561d85a9d2aa76840
Author: Hyukjin Kwon <gurwls...@apache.org>
AuthorDate: Sat Apr 6 10:54:27 2024 -0700

    [SPARK-47727][PYTHON] Make SparkConf to root level to for both SparkSession 
and SparkContext
    
    ### What changes were proposed in this pull request?
    
    This PR proposes to make SparkConf to root level to for both `SparkSession` 
and `SparkContext`.
    
    ### Why are the changes needed?
    
    `SparkConf` is special. `SparkSession.builder.options` can take it as an 
option, and this instance can be created without JVM access. So it can be 
shared with pure Python `pysaprk-connect` package.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    
    CI in this PR should verify them.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No.
    
    Closes #45873 from HyukjinKwon/SPARK-47727.
    
    Lead-authored-by: Hyukjin Kwon <gurwls...@apache.org>
    Co-authored-by: Hyukjin Kwon <gurwls...@gmail.com>
    Signed-off-by: Dongjoon Hyun <dh...@apple.com>
---
 dev/sparktestsupport/modules.py          |  2 +-
 python/pyspark/__init__.py               |  5 ++---
 python/pyspark/{core => }/conf.py        | 37 ++++++++++++++++++--------------
 python/pyspark/core/context.py           |  3 +--
 python/pyspark/sql/session.py            | 10 ++++-----
 python/pyspark/sql/tests/test_session.py |  3 ---
 python/pyspark/testing/utils.py          |  6 +++++-
 7 files changed, 34 insertions(+), 32 deletions(-)

diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index d3ffa79ebe68..701203414702 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -430,9 +430,9 @@ pyspark_core = Module(
     source_file_regexes=["python/(?!pyspark/(ml|mllib|sql|streaming))"],
     python_test_goals=[
         # doctests
+        "pyspark.conf",
         "pyspark.core.rdd",
         "pyspark.core.context",
-        "pyspark.core.conf",
         "pyspark.core.broadcast",
         "pyspark.accumulators",
         "pyspark.core.files",
diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py
index 032da1857a87..15c21df0c6bf 100644
--- a/python/pyspark/__init__.py
+++ b/python/pyspark/__init__.py
@@ -53,20 +53,19 @@ from typing import cast, Any, Callable, TypeVar, Union
 from pyspark.util import is_remote_only
 
 if not is_remote_only():
-    from pyspark.core.conf import SparkConf
     from pyspark.core.rdd import RDD, RDDBarrier
     from pyspark.core.files import SparkFiles
     from pyspark.core.status import StatusTracker, SparkJobInfo, SparkStageInfo
     from pyspark.core.broadcast import Broadcast
-    from pyspark.core import conf, rdd, files, status, broadcast
+    from pyspark.core import rdd, files, status, broadcast
 
     # for backward compatibility references.
-    sys.modules["pyspark.conf"] = conf
     sys.modules["pyspark.rdd"] = rdd
     sys.modules["pyspark.files"] = files
     sys.modules["pyspark.status"] = status
     sys.modules["pyspark.broadcast"] = broadcast
 
+from pyspark.conf import SparkConf
 from pyspark.util import InheritableThread, inheritable_thread_target
 from pyspark.storagelevel import StorageLevel
 from pyspark.accumulators import Accumulator, AccumulatorParam
diff --git a/python/pyspark/core/conf.py b/python/pyspark/conf.py
similarity index 90%
rename from python/pyspark/core/conf.py
rename to python/pyspark/conf.py
index fe7879c3501b..ca03266a11c6 100644
--- a/python/pyspark/core/conf.py
+++ b/python/pyspark/conf.py
@@ -18,12 +18,14 @@
 __all__ = ["SparkConf"]
 
 import sys
-from typing import Dict, List, Optional, Tuple, cast, overload
-
-from py4j.java_gateway import JVMView, JavaObject
+from typing import Dict, List, Optional, Tuple, cast, overload, TYPE_CHECKING
 
+from pyspark.util import is_remote_only
 from pyspark.errors import PySparkRuntimeError
 
+if TYPE_CHECKING:
+    from py4j.java_gateway import JVMView, JavaObject
+
 
 class SparkConf:
     """
@@ -60,11 +62,10 @@ class SparkConf:
 
     Examples
     --------
-    >>> from pyspark.core.conf import SparkConf
-    >>> from pyspark.core.context import SparkContext
+    >>> from pyspark import SparkConf, SparkContext
     >>> conf = SparkConf()
     >>> conf.setMaster("local").setAppName("My app")
-    <pyspark.core.conf.SparkConf object at ...>
+    <pyspark.conf.SparkConf object at ...>
     >>> conf.get("spark.master")
     'local'
     >>> conf.get("spark.app.name")
@@ -79,13 +80,13 @@ class SparkConf:
 
     >>> conf = SparkConf(loadDefaults=False)
     >>> conf.setSparkHome("/path")
-    <pyspark.core.conf.SparkConf object at ...>
+    <pyspark.conf.SparkConf object at ...>
     >>> conf.get("spark.home")
     '/path'
     >>> conf.setExecutorEnv("VAR1", "value1")
-    <pyspark.core.conf.SparkConf object at ...>
+    <pyspark.conf.SparkConf object at ...>
     >>> conf.setExecutorEnv(pairs = [("VAR3", "value3"), ("VAR4", "value4")])
-    <pyspark.core.conf.SparkConf object at ...>
+    <pyspark.conf.SparkConf object at ...>
     >>> conf.get("spark.executorEnv.VAR1")
     'value1'
     >>> print(conf.toDebugString())
@@ -109,14 +110,14 @@ class SparkConf:
     spark.home=/path
     """
 
-    _jconf: Optional[JavaObject]
+    _jconf: Optional["JavaObject"]
     _conf: Optional[Dict[str, str]]
 
     def __init__(
         self,
         loadDefaults: bool = True,
-        _jvm: Optional[JVMView] = None,
-        _jconf: Optional[JavaObject] = None,
+        _jvm: Optional["JVMView"] = None,
+        _jconf: Optional["JavaObject"] = None,
     ):
         """
         Create a new Spark configuration.
@@ -124,13 +125,15 @@ class SparkConf:
         if _jconf:
             self._jconf = _jconf
         else:
-            from pyspark.core.context import SparkContext
+            jvm = None
+            if not is_remote_only():
+                from pyspark.core.context import SparkContext
 
-            _jvm = _jvm or SparkContext._jvm
+                jvm = _jvm or SparkContext._jvm
 
-            if _jvm is not None:
+            if jvm is not None:
                 # JVM is created, so create self._jconf directly through JVM
-                self._jconf = _jvm.SparkConf(loadDefaults)
+                self._jconf = jvm.SparkConf(loadDefaults)
                 self._conf = None
             else:
                 # JVM is not created, so store data in self._conf first
@@ -240,6 +243,8 @@ class SparkConf:
     def getAll(self) -> List[Tuple[str, str]]:
         """Get all values as a list of key-value pairs."""
         if self._jconf is not None:
+            from py4j.java_gateway import JavaObject
+
             return [(elem._1(), elem._2()) for elem in cast(JavaObject, 
self._jconf).getAll()]
         else:
             assert self._conf is not None
diff --git a/python/pyspark/core/context.py b/python/pyspark/core/context.py
index 076119b959b1..be2f103e7a94 100644
--- a/python/pyspark/core/context.py
+++ b/python/pyspark/core/context.py
@@ -47,9 +47,9 @@ from py4j.java_collections import JavaMap
 from py4j.protocol import Py4JError
 
 from pyspark import accumulators
+from pyspark.conf import SparkConf
 from pyspark.accumulators import Accumulator
 from pyspark.core.broadcast import Broadcast, BroadcastPickleRegistry
-from pyspark.core.conf import SparkConf
 from pyspark.core.files import SparkFiles
 from pyspark.java_gateway import launch_gateway
 from pyspark.serializers import (
@@ -2598,7 +2598,6 @@ class SparkContext:
 
 def _test() -> None:
     import doctest
-    from pyspark import SparkConf
 
     globs = globals().copy()
     conf = SparkConf().set("spark.ui.enabled", "True")
diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
index 0cb02b22a60d..11e0ef43b59f 100644
--- a/python/pyspark/sql/session.py
+++ b/python/pyspark/sql/session.py
@@ -38,6 +38,7 @@ from typing import (
     TYPE_CHECKING,
 )
 
+from pyspark.conf import SparkConf
 from pyspark.util import is_remote_only
 from pyspark.sql.column import _to_java_column
 from pyspark.sql.conf import RuntimeConfig
@@ -67,7 +68,6 @@ from pyspark.errors import PySparkValueError, 
PySparkTypeError, PySparkRuntimeEr
 
 if TYPE_CHECKING:
     from py4j.java_gateway import JavaObject
-    from pyspark.core.conf import SparkConf
     from pyspark.core.context import SparkContext
     from pyspark.core.rdd import RDD
     from pyspark.sql._typing import AtomicValue, RowLike, OptionalPrimitiveType
@@ -221,7 +221,7 @@ class SparkSession(SparkConversionMixin):
             self._options: Dict[str, Any] = {}
 
         @overload
-        def config(self, *, conf: "SparkConf") -> "SparkSession.Builder":
+        def config(self, *, conf: SparkConf) -> "SparkSession.Builder":
             ...
 
         @overload
@@ -236,7 +236,7 @@ class SparkSession(SparkConversionMixin):
             self,
             key: Optional[str] = None,
             value: Optional[Any] = None,
-            conf: Optional["SparkConf"] = None,
+            conf: Optional[SparkConf] = None,
             *,
             map: Optional[Dict[str, "OptionalPrimitiveType"]] = None,
         ) -> "SparkSession.Builder":
@@ -273,7 +273,7 @@ class SparkSession(SparkConversionMixin):
             --------
             For an existing :class:`SparkConf`, use `conf` parameter.
 
-            >>> from pyspark.core.conf import SparkConf
+            >>> from pyspark.conf import SparkConf
             >>> conf = SparkConf().setAppName("example").setMaster("local")
             >>> SparkSession.builder.config(conf=conf)
             <pyspark.sql.session.SparkSession.Builder...
@@ -489,7 +489,6 @@ class SparkSession(SparkConversionMixin):
                 return 
RemoteSparkSession.builder.config(map=opts).getOrCreate()  # type: ignore
 
             from pyspark.core.context import SparkContext
-            from pyspark.core.conf import SparkConf
 
             with self._lock:
                 if (
@@ -1213,7 +1212,6 @@ class SparkSession(SparkConversionMixin):
         that script, which would expose those to users.
         """
         import py4j
-        from pyspark.core.conf import SparkConf
         from pyspark.core.context import SparkContext
 
         try:
diff --git a/python/pyspark/sql/tests/test_session.py 
b/python/pyspark/sql/tests/test_session.py
index ce2c6bf3a1df..d37f0aa4a601 100644
--- a/python/pyspark/sql/tests/test_session.py
+++ b/python/pyspark/sql/tests/test_session.py
@@ -193,9 +193,6 @@ class SparkSessionTests3(unittest.TestCase, 
PySparkErrorTestUtils):
             session.range(5).collect()
 
     def test_active_session_with_None_and_not_None_context(self):
-        from pyspark.core.context import SparkContext
-        from pyspark.core.conf import SparkConf
-
         sc = None
         session = None
         try:
diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py
index 934d7d872e20..de40685dedc0 100644
--- a/python/pyspark/testing/utils.py
+++ b/python/pyspark/testing/utils.py
@@ -52,7 +52,7 @@ except ImportError:
     # No NumPy, but that's okay, we'll skip those tests
     pass
 
-from pyspark import SparkContext, SparkConf
+from pyspark import SparkConf
 from pyspark.errors import PySparkAssertionError, PySparkException
 from pyspark.find_spark_home import _find_spark_home
 from pyspark.sql.dataframe import DataFrame
@@ -154,6 +154,8 @@ class QuietTest:
 
 class PySparkTestCase(unittest.TestCase):
     def setUp(self):
+        from pyspark import SparkContext
+
         self._old_sys_path = list(sys.path)
         class_name = self.__class__.__name__
         self.sc = SparkContext("local[4]", class_name)
@@ -173,6 +175,8 @@ class ReusedPySparkTestCase(unittest.TestCase):
 
     @classmethod
     def setUpClass(cls):
+        from pyspark import SparkContext
+
         cls.sc = SparkContext("local[4]", cls.__name__, conf=cls.conf())
 
     @classmethod


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

Reply via email to