diff options
Diffstat (limited to 'python/pyspark/sql')
-rw-r--r-- | python/pyspark/sql/context.py | 5 | ||||
-rw-r--r-- | python/pyspark/sql/session.py | 17 |
2 files changed, 18 insertions, 4 deletions
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index e8e60c6412..486733a390 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -34,7 +34,10 @@ __all__ = ["SQLContext", "HiveContext", "UDFRegistration"] class SQLContext(object): - """Wrapper around :class:`SparkSession`, the main entry point to Spark SQL functionality. + """The entry point for working with structured data (rows and columns) in Spark, in Spark 1.x. + + As of Spark 2.0, this is replaced by :class:`SparkSession`. However, we are keeping the class + here for backward compatibility. A SQLContext can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. 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() |