aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--python/pyspark/sql/session.py35
1 files changed, 24 insertions, 11 deletions
diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
index 241947537f..52e7f3d348 100644
--- a/python/pyspark/sql/session.py
+++ b/python/pyspark/sql/session.py
@@ -138,24 +138,37 @@ class SparkSession(object):
"""Gets an existing :class:`SparkSession` or, if there is no existing one, creates a
new one based on the options set in this builder.
- This method first checks whether there is a valid thread-local SparkSession,
- and if yes, return that one. It then checks whether there is a valid global
- default SparkSession, and if yes, return that one. If no valid global default
- SparkSession exists, the method creates a new SparkSession and assigns the
- newly created SparkSession as the global default.
+ This method first checks whether there is a valid global default SparkSession, and if
+ yes, return that one. If no valid global default SparkSession exists, the method
+ creates a new SparkSession and assigns the newly created SparkSession as the global
+ default.
+
+ >>> s1 = SparkSession.builder.config("k1", "v1").getOrCreate()
+ >>> s1.conf.get("k1") == "v1"
+ True
In case an existing SparkSession is returned, the config options specified
in this builder will be applied to the existing SparkSession.
+
+ >>> s2 = SparkSession.builder.config("k2", "v2").getOrCreate()
+ >>> s1.conf.get("k1") == s2.conf.get("k1")
+ True
+ >>> s1.conf.get("k2") == s2.conf.get("k2")
+ True
"""
with self._lock:
- from pyspark.conf import SparkConf
from pyspark.context import SparkContext
- from pyspark.sql.context import SQLContext
- sparkConf = SparkConf()
+ from pyspark.conf import SparkConf
+ session = SparkSession._instantiatedContext
+ if session is None:
+ sparkConf = SparkConf()
+ for key, value in self._options.items():
+ sparkConf.set(key, value)
+ sc = SparkContext.getOrCreate(sparkConf)
+ session = SparkSession(sc)
for key, value in self._options.items():
- sparkConf.set(key, value)
- sparkContext = SparkContext.getOrCreate(sparkConf)
- return SQLContext.getOrCreate(sparkContext).sparkSession
+ session.conf.set(key, value)
+ return session
builder = Builder()