aboutsummaryrefslogtreecommitdiff
path: root/python
diff options
context:
space:
mode:
authorVinayak <vijoshi5@in.ibm.com>2017-01-13 18:35:12 +0800
committerWenchen Fan <wenchen@databricks.com>2017-01-13 18:35:51 +0800
commit285a7798e267311730b0163d37d726a81465468a (patch)
treecf6517c6149b46d3360c5b30a979429330c7632f /python
parentb040cef2ed0ed46c3dfb483a117200c9dac074ca (diff)
downloadspark-285a7798e267311730b0163d37d726a81465468a.tar.gz
spark-285a7798e267311730b0163d37d726a81465468a.tar.bz2
spark-285a7798e267311730b0163d37d726a81465468a.zip
[SPARK-18687][PYSPARK][SQL] Backward compatibility - creating a Dataframe on a new SQLContext object fails with a Derby error
Change is for SQLContext to reuse the active SparkSession during construction if the sparkContext supplied is the same as the currently active SparkContext. Without this change, a new SparkSession is instantiated that results in a Derby error when attempting to create a dataframe using a new SQLContext object even though the SparkContext supplied to the new SQLContext is same as the currently active one. Refer https://issues.apache.org/jira/browse/SPARK-18687 for details on the error and a repro. Existing unit tests and a new unit test added to pyspark-sql: /python/run-tests --python-executables=python --modules=pyspark-sql Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Vinayak <vijoshi5@in.ibm.com> Author: Vinayak Joshi <vijoshi@users.noreply.github.com> Closes #16119 from vijoshi/SPARK-18687_master.
Diffstat (limited to 'python')
-rw-r--r--python/pyspark/sql/context.py2
-rw-r--r--python/pyspark/sql/tests.py7
2 files changed, 7 insertions, 2 deletions
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index de4c335ad2..c22f4b87e1 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -73,7 +73,7 @@ class SQLContext(object):
self._jsc = self._sc._jsc
self._jvm = self._sc._jvm
if sparkSession is None:
- sparkSession = SparkSession(sparkContext)
+ sparkSession = SparkSession.builder.getOrCreate()
if jsqlContext is None:
jsqlContext = sparkSession._jwrapped
self.sparkSession = sparkSession
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index d1782857e6..a8250281da 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -47,7 +47,7 @@ else:
import unittest
from pyspark import SparkContext
-from pyspark.sql import SparkSession, HiveContext, Column, Row
+from pyspark.sql import SparkSession, SQLContext, HiveContext, Column, Row
from pyspark.sql.types import *
from pyspark.sql.types import UserDefinedType, _infer_type
from pyspark.tests import ReusedPySparkTestCase, SparkSubmitTests
@@ -206,6 +206,11 @@ class SQLTests(ReusedPySparkTestCase):
cls.spark.stop()
shutil.rmtree(cls.tempdir.name, ignore_errors=True)
+ def test_sqlcontext_reuses_sparksession(self):
+ sqlContext1 = SQLContext(self.sc)
+ sqlContext2 = SQLContext(self.sc)
+ self.assertTrue(sqlContext1.sparkSession is sqlContext2.sparkSession)
+
def test_row_should_be_read_only(self):
row = Row(a=1, b=2)
self.assertEqual(1, row.a)