spark git commit: [SPARK-18687][PYSPARK][SQL] Backward compatibility - creating a Dataframe on a new SQLContext object fails with a Derby error
Repository: spark Updated Branches: refs/heads/branch-2.0 be527ddc0 -> 449231c65 [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: VinayakAuthor: Vinayak Joshi Closes #16119 from vijoshi/SPARK-18687_master. (cherry picked from commit 285a7798e267311730b0163d37d726a81465468a) Signed-off-by: Wenchen Fan Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/449231c6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/449231c6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/449231c6 Branch: refs/heads/branch-2.0 Commit: 449231c65c92ecff0ac8d3cee4bf5e652f1fe87e Parents: be527dd Author: Vinayak Authored: Fri Jan 13 18:35:12 2017 +0800 Committer: Wenchen Fan Committed: Fri Jan 13 18:36:35 2017 +0800 -- python/pyspark/sql/context.py | 2 +- python/pyspark/sql/tests.py | 7 ++- 2 files changed, 7 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/449231c6/python/pyspark/sql/context.py -- diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 8cdf371..f6f5e8b 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 http://git-wip-us.apache.org/repos/asf/spark/blob/449231c6/python/pyspark/sql/tests.py -- diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 796b964..2c6b336 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 @@ -201,6 +201,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) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-18687][PYSPARK][SQL] Backward compatibility - creating a Dataframe on a new SQLContext object fails with a Derby error
Repository: spark Updated Branches: refs/heads/branch-2.1 0668e061b -> b2c9a2c8c [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: VinayakAuthor: Vinayak Joshi Closes #16119 from vijoshi/SPARK-18687_master. (cherry picked from commit 285a7798e267311730b0163d37d726a81465468a) Signed-off-by: Wenchen Fan Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b2c9a2c8 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b2c9a2c8 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b2c9a2c8 Branch: refs/heads/branch-2.1 Commit: b2c9a2c8c8e8c38baa6d876c81d143af61328aa2 Parents: 0668e06 Author: Vinayak Authored: Fri Jan 13 18:35:12 2017 +0800 Committer: Wenchen Fan Committed: Fri Jan 13 18:36:17 2017 +0800 -- python/pyspark/sql/context.py | 2 +- python/pyspark/sql/tests.py | 7 ++- 2 files changed, 7 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b2c9a2c8/python/pyspark/sql/context.py -- diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index de4c335..c22f4b8 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 http://git-wip-us.apache.org/repos/asf/spark/blob/b2c9a2c8/python/pyspark/sql/tests.py -- diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index fe034bc..20b9351 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) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-18687][PYSPARK][SQL] Backward compatibility - creating a Dataframe on a new SQLContext object fails with a Derby error
Repository: spark Updated Branches: refs/heads/master b040cef2e -> 285a7798e [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: VinayakAuthor: Vinayak Joshi Closes #16119 from vijoshi/SPARK-18687_master. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/285a7798 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/285a7798 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/285a7798 Branch: refs/heads/master Commit: 285a7798e267311730b0163d37d726a81465468a Parents: b040cef Author: Vinayak Authored: Fri Jan 13 18:35:12 2017 +0800 Committer: Wenchen Fan Committed: Fri Jan 13 18:35:51 2017 +0800 -- python/pyspark/sql/context.py | 2 +- python/pyspark/sql/tests.py | 7 ++- 2 files changed, 7 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/285a7798/python/pyspark/sql/context.py -- diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index de4c335..c22f4b8 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 http://git-wip-us.apache.org/repos/asf/spark/blob/285a7798/python/pyspark/sql/tests.py -- diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index d178285..a825028 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) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org