From 66773eb8a55bfe6437dd4096c2c55685aca29dcd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 29 Apr 2016 20:46:07 -0700 Subject: [SPARK-15012][SQL] Simplify configuration API further ## What changes were proposed in this pull request? 1. Remove all the `spark.setConf` etc. Just expose `spark.conf` 2. Make `spark.conf` take in things set in the core `SparkConf` as well, otherwise users may get confused This was done for both the Python and Scala APIs. ## How was this patch tested? `SQLConfSuite`, python tests. This one fixes the failed tests in #12787 Closes #12787 Author: Andrew Or Author: Yin Huai Closes #12798 from yhuai/conf-api. --- python/pyspark/sql/context.py | 4 ++-- python/pyspark/sql/session.py | 29 ----------------------------- python/pyspark/sql/tests.py | 4 ++-- 3 files changed, 4 insertions(+), 33 deletions(-) (limited to 'python/pyspark') diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 417d719c35..2096236d7f 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -114,7 +114,7 @@ class SQLContext(object): def setConf(self, key, value): """Sets the given Spark SQL configuration property. """ - self.sparkSession.setConf(key, value) + self.sparkSession.conf.set(key, value) @ignore_unicode_prefix @since(1.3) @@ -133,7 +133,7 @@ class SQLContext(object): >>> sqlContext.getConf("spark.sql.shuffle.partitions", u"10") u'50' """ - return self.sparkSession.getConf(key, defaultValue) + return self.sparkSession.conf.get(key, defaultValue) @property @since("1.3.1") diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index c2452613ba..35c36b4935 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -134,35 +134,6 @@ class SparkSession(object): self._conf = RuntimeConfig(self._jsparkSession.conf()) return self._conf - @since(2.0) - def setConf(self, key, value): - """ - Sets the given Spark SQL configuration property. - """ - self._jsparkSession.setConf(key, value) - - @ignore_unicode_prefix - @since(2.0) - def getConf(self, key, defaultValue=None): - """Returns the value of Spark SQL configuration property for the given key. - - If the key is not set and defaultValue is not None, return - defaultValue. If the key is not set and defaultValue is None, return - the system default value. - - >>> spark.getConf("spark.sql.shuffle.partitions") - u'200' - >>> spark.getConf("spark.sql.shuffle.partitions", "10") - u'10' - >>> spark.setConf("spark.sql.shuffle.partitions", "50") - >>> spark.getConf("spark.sql.shuffle.partitions", "10") - u'50' - """ - if defaultValue is not None: - return self._jsparkSession.getConf(key, defaultValue) - else: - return self._jsparkSession.getConf(key) - @property @since(2.0) def catalog(self): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index ea98206836..4995b263e1 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1397,9 +1397,9 @@ class SQLTests(ReusedPySparkTestCase): def test_conf(self): spark = self.sparkSession - spark.setConf("bogo", "sipeo") + spark.conf.set("bogo", "sipeo") self.assertEqual(self.sparkSession.conf.get("bogo"), "sipeo") - spark.setConf("bogo", "ta") + spark.conf.set("bogo", "ta") self.assertEqual(spark.conf.get("bogo"), "ta") self.assertEqual(spark.conf.get("bogo", "not.read"), "ta") self.assertEqual(spark.conf.get("not.set", "ta"), "ta") -- cgit v1.2.3