aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql/session.py
diff options
context:
space:
mode:
authorReynold Xin <rxin@databricks.com>2016-05-19 21:53:26 -0700
committerReynold Xin <rxin@databricks.com>2016-05-19 21:53:26 -0700
commitf2ee0ed4b7ecb2855cc4928a9613a07d45446f4e (patch)
tree3c923b935bcf35219f158ed5a8ca34edfb7c9322 /python/pyspark/sql/session.py
parent17591d90e6873f30a042112f56a1686726ccbd60 (diff)
downloadspark-f2ee0ed4b7ecb2855cc4928a9613a07d45446f4e.tar.gz
spark-f2ee0ed4b7ecb2855cc4928a9613a07d45446f4e.tar.bz2
spark-f2ee0ed4b7ecb2855cc4928a9613a07d45446f4e.zip
[SPARK-15075][SPARK-15345][SQL] Clean up SparkSession builder and propagate config options to existing sessions if specified
## What changes were proposed in this pull request? Currently SparkSession.Builder use SQLContext.getOrCreate. It should probably the the other way around, i.e. all the core logic goes in SparkSession, and SQLContext just calls that. This patch does that. This patch also makes sure config options specified in the builder are propagated to the existing (and of course the new) SparkSession. ## How was this patch tested? Updated tests to reflect the change, and also introduced a new SparkSessionBuilderSuite that should cover all the branches. Author: Reynold Xin <rxin@databricks.com> Closes #13200 from rxin/SPARK-15075.
Diffstat (limited to 'python/pyspark/sql/session.py')
-rw-r--r--python/pyspark/sql/session.py17
1 files changed, 14 insertions, 3 deletions
diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py
index 257a239c8d..0e04b88265 100644
--- a/python/pyspark/sql/session.py
+++ b/python/pyspark/sql/session.py
@@ -120,6 +120,8 @@ class SparkSession(object):
def appName(self, name):
"""Sets a name for the application, which will be shown in the Spark web UI.
+ If no application name is set, a randomly generated name will be used.
+
:param name: an application name
"""
return self.config("spark.app.name", name)
@@ -133,8 +135,17 @@ class SparkSession(object):
@since(2.0)
def getOrCreate(self):
- """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a new
- one based on the options set in this builder.
+ """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.
+
+ In case an existing SparkSession is returned, the config options specified
+ in this builder will be applied to the existing SparkSession.
"""
with self._lock:
from pyspark.conf import SparkConf
@@ -175,7 +186,7 @@ class SparkSession(object):
if jsparkSession is None:
jsparkSession = self._jvm.SparkSession(self._jsc.sc())
self._jsparkSession = jsparkSession
- self._jwrapped = self._jsparkSession.wrapped()
+ self._jwrapped = self._jsparkSession.sqlContext()
self._wrapped = SQLContext(self._sc, self, self._jwrapped)
_monkey_patch_RDD(self)
install_exception_handler()