From 9de836ece3800af0cecf29b130d3ef50471c130a Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 24 Mar 2020 21:26:24 +0800 Subject: [PATCH 01/10] [SPARK-31234][SQL] ResetCommand should reset config to sc.conf only --- .../apache/spark/sql/internal/SQLConf.scala | 2 ++ .../sql/execution/command/SetCommand.scala | 6 +++++- .../spark/sql/internal/SQLConfSuite.scala | 20 +++++++++---------- .../spark/sql/test/SharedSparkSession.scala | 1 + 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9a524defb281..4c567dcae82a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3258,7 +3258,9 @@ class SQLConf extends Serializable with Logging { } def clear(): Unit = { + val reserved = settings.asScala.filterKeys(staticConfKeys.contains).toMap settings.clear() + settings.putAll(reserved.asJava) } override def clone(): SQLConf = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index c55ff4ffefa0..8066f8402134 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -170,7 +170,11 @@ object SetCommand { case object ResetCommand extends RunnableCommand with IgnoreCachedData { override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.sessionState.conf.clear() + val conf = sparkSession.sessionState.conf + conf.clear() + sparkSession.sparkContext.conf.getAll.foreach { case (k, v) => + conf.setConfString(k, v) + } Seq.empty[Row] } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 650dc127a060..ec09a3f332cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -32,20 +32,13 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { private val testKey = "test.key.0" private val testVal = "test.val.0" - test("propagate from spark conf") { - // We create a new context here to avoid order dependence with other tests that might call - // clear(). - val newContext = new SQLContext(SparkSession.builder().sparkContext(sparkContext).getOrCreate()) - assert(newContext.getConf("spark.sql.testkey", "false") === "true") - } - test("programmatic ways of basic setting and getting") { // Set a conf first. spark.conf.set(testKey, testVal) // Clear the conf. spark.sessionState.conf.clear() // After clear, only overrideConfs used by unit test should be in the SQLConf. - assert(spark.conf.getAll === TestSQLContext.overrideConfs) + assert((spark.conf.getAll -- TestSQLContext.overrideConfs.keys).size < spark.conf.getAll.size) spark.conf.set(testKey, testVal) assert(spark.conf.get(testKey) === testVal) @@ -115,6 +108,13 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } } + test("reset - static conf") { + spark.sessionState.conf.clear() + assert(spark.conf.get(StaticSQLConf.UI_RETAINED_EXECUTIONS) === 1) + sql(s"reset") + assert(spark.conf.get(StaticSQLConf.UI_RETAINED_EXECUTIONS) === 1) + } + test("reset - public conf") { spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) @@ -218,8 +218,8 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { test("default value of WAREHOUSE_PATH") { // JVM adds a trailing slash if the directory exists and leaves it as-is, if it doesn't // In our comparison, strip trailing slash off of both sides, to account for such cases - assert(new Path(Utils.resolveURI("spark-warehouse")).toString.stripSuffix("/") === spark - .sessionState.conf.warehousePath.stripSuffix("/")) + assert(new Path(Utils.resolveURI(s"spark-warehouse/${classOf[SQLConfSuite].getCanonicalName}")) + .toString.stripSuffix("/") === spark.sessionState.conf.warehousePath.stripSuffix("/")) } test("static SQL conf comes from SparkConf") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala index ee29b4b8fb32..2b81c29df10f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -72,6 +72,7 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) + .set(StaticSQLConf.UI_RETAINED_EXECUTIONS, 1) conf.set( StaticSQLConf.WAREHOUSE_PATH, conf.get(StaticSQLConf.WAREHOUSE_PATH) + "/" + getClass.getCanonicalName) From 3b6b3aab3b42d0aff5f8b0b1785164da0a154063 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 25 Mar 2020 00:24:52 +0800 Subject: [PATCH 02/10] nit --- .../scala/org/apache/spark/sql/internal/SQLConfSuite.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index ec09a3f332cc..18a22e0ba852 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -32,6 +32,13 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { private val testKey = "test.key.0" private val testVal = "test.val.0" + test("propagate from spark conf") { + // We create a new context here to avoid order dependence with other tests that might call + // clear(). + val newContext = new SQLContext(SparkSession.builder().sparkContext(sparkContext).getOrCreate()) + assert(newContext.getConf("spark.sql.testkey", "false") === "true") + } + test("programmatic ways of basic setting and getting") { // Set a conf first. spark.conf.set(testKey, testVal) From 435e4112355a31c158619dd2ca4f05d6d71242f9 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 25 Mar 2020 13:18:57 +0800 Subject: [PATCH 03/10] fix tests --- docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 2 +- .../sql/execution/command/SetCommand.scala | 3 ++- .../org/apache/spark/sql/SQLQuerySuite.scala | 22 +++++++------------ .../spark/sql/internal/SQLConfSuite.scala | 16 ++++++++++---- .../spark/sql/test/SharedSparkSession.scala | 1 - 5 files changed, 23 insertions(+), 21 deletions(-) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index 8ee61514ee4e..c1aaafeb1e48 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -20,7 +20,7 @@ license: | --- ### Description -Reset all the properties specific to the current session to their default values. After RESET command, executing SET command will output empty. +Reset any properties specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. ### Syntax {% highlight sql %} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 8066f8402134..451ddc32b567 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -162,7 +162,8 @@ object SetCommand { } /** - * This command is for resetting SQLConf to the default values. Command that runs + * This command is for resetting SQLConf to the default values. Any configurations that were set + * via [[SetCommand]] will get reset to default value Command that runs * {{{ * reset; * }}} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index de0f7801a39a..cef066355686 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1021,36 +1021,30 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } test("SET commands semantics using sql()") { - spark.sessionState.conf.clear() val testKey = "test.key.0" val testVal = "test.val.0" val nonexistentKey = "nonexistent" // "set" itself returns all config variables currently specified in SQLConf. - assert(sql("SET").collect().size === TestSQLContext.overrideConfs.size) - sql("SET").collect().foreach { row => - val key = row.getString(0) - val value = row.getString(1) - assert( - TestSQLContext.overrideConfs.contains(key), - s"$key should exist in SQLConf.") - assert( - TestSQLContext.overrideConfs(key) === value, - s"The value of $key should be ${TestSQLContext.overrideConfs(key)} instead of $value.") + TestSQLContext.overrideConfs.foreach { case (k, _) => + assert(sql("SET").where(s"key ='$k'").collect().head.get(1) === + TestSQLContext.overrideConfs(k)) + } - val overrideConfs = sql("SET").collect() + + val originalConfs = sql("SET").collect() // "set key=val" sql(s"SET $testKey=$testVal") checkAnswer( sql("SET"), - overrideConfs ++ Seq(Row(testKey, testVal)) + originalConfs ++ Seq(Row(testKey, testVal)) ) sql(s"SET ${testKey + testKey}=${testVal + testVal}") checkAnswer( sql("set"), - overrideConfs ++ Seq(Row(testKey, testVal), Row(testKey + testKey, testVal + testVal)) + originalConfs ++ Seq(Row(testKey, testVal), Row(testKey + testKey, testVal + testVal)) ) // "set key" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 18a22e0ba852..17d4fcacac8e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -115,11 +115,19 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } } - test("reset - static conf") { - spark.sessionState.conf.clear() - assert(spark.conf.get(StaticSQLConf.UI_RETAINED_EXECUTIONS) === 1) + test("reset - static and spark conf") { + val conf = spark.sessionState.conf.getAllConfs + val appName = conf.get("spark.app.name") + val driverHost = conf.get("spark.driver.host") + val master = conf.get("spark.master") + val warehouseDir = conf.get("spark.sql.warehouse.dir") + // ensure the conf here is not default value, and will not be reset to default value + assert(warehouseDir.get.contains(this.getClass.getCanonicalName)) sql(s"reset") - assert(spark.conf.get(StaticSQLConf.UI_RETAINED_EXECUTIONS) === 1) + assert(conf.get("spark.app.name") === appName) + assert(conf.get("spark.driver.host") === driverHost) + assert(conf.get("spark.master") === master) + assert(conf.get("spark.sql.warehouse.dir") === warehouseDir) } test("reset - public conf") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala index 2b81c29df10f..ee29b4b8fb32 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -72,7 +72,6 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) - .set(StaticSQLConf.UI_RETAINED_EXECUTIONS, 1) conf.set( StaticSQLConf.WAREHOUSE_PATH, conf.get(StaticSQLConf.WAREHOUSE_PATH) + "/" + getClass.getCanonicalName) From ef636045025beee11d1688be72381f413ca0bb64 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 25 Mar 2020 14:38:22 +0800 Subject: [PATCH 04/10] nnit --- docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 2 +- .../org/apache/spark/sql/execution/command/SetCommand.scala | 2 +- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 1 - 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index c1aaafeb1e48..34c243d3be9e 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -20,7 +20,7 @@ license: | --- ### Description -Reset any properties specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. +Reset any runtime properties specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. ### Syntax {% highlight sql %} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 451ddc32b567..3dc1d5269771 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -163,7 +163,7 @@ object SetCommand { /** * This command is for resetting SQLConf to the default values. Any configurations that were set - * via [[SetCommand]] will get reset to default value Command that runs + * via [[SetCommand]] will get reset to default value. Command that runs * {{{ * reset; * }}} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index cef066355686..477f195df311 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1029,7 +1029,6 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark TestSQLContext.overrideConfs.foreach { case (k, _) => assert(sql("SET").where(s"key ='$k'").collect().head.get(1) === TestSQLContext.overrideConfs(k)) - } val originalConfs = sql("SET").collect() From bcc678656ae134e5917e244a4fa6662920b662ab Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 25 Mar 2020 14:43:56 +0800 Subject: [PATCH 05/10] nnit --- docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index 34c243d3be9e..085f21b638bd 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -20,7 +20,7 @@ license: | --- ### Description -Reset any runtime properties specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. +Reset any runtime configurations specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values. ### Syntax {% highlight sql %} From 69a4b82422b53fe4345419615d46782283119bc9 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 25 Mar 2020 18:13:46 +0800 Subject: [PATCH 06/10] address cmts --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 2 -- .../org/apache/spark/sql/internal/SQLConfSuite.scala | 8 ++++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4c567dcae82a..9a524defb281 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -3258,9 +3258,7 @@ class SQLConf extends Serializable with Logging { } def clear(): Unit = { - val reserved = settings.asScala.filterKeys(staticConfKeys.contains).toMap settings.clear() - settings.putAll(reserved.asJava) } override def clone(): SQLConf = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 17d4fcacac8e..c244e3bf7621 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -116,12 +116,12 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } test("reset - static and spark conf") { - val conf = spark.sessionState.conf.getAllConfs + val conf = spark.sparkContext.getConf.getAll.toMap val appName = conf.get("spark.app.name") val driverHost = conf.get("spark.driver.host") val master = conf.get("spark.master") val warehouseDir = conf.get("spark.sql.warehouse.dir") - // ensure the conf here is not default value, and will not be reset to default value + // ensure the conf here is not default value, and will not be reset to default value later assert(warehouseDir.get.contains(this.getClass.getCanonicalName)) sql(s"reset") assert(conf.get("spark.app.name") === appName) @@ -233,8 +233,8 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { test("default value of WAREHOUSE_PATH") { // JVM adds a trailing slash if the directory exists and leaves it as-is, if it doesn't // In our comparison, strip trailing slash off of both sides, to account for such cases - assert(new Path(Utils.resolveURI(s"spark-warehouse/${classOf[SQLConfSuite].getCanonicalName}")) - .toString.stripSuffix("/") === spark.sessionState.conf.warehousePath.stripSuffix("/")) + assert(new Path(Utils.resolveURI("spark-warehouse")).toString.stripSuffix("/") === + spark.sessionState.conf.warehousePath.stripSuffix("/")) } test("static SQL conf comes from SparkConf") { From 460c35b940b040f414f01ff8156c46ff02646888 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 25 Mar 2020 19:06:05 +0800 Subject: [PATCH 07/10] unnecessary --- .../org/apache/spark/sql/SQLQuerySuite.scala | 19 ++++++++++++------- .../spark/sql/internal/SQLConfSuite.scala | 6 +++--- 2 files changed, 15 insertions(+), 10 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 477f195df311..15f53a203d68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1021,29 +1021,34 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } test("SET commands semantics using sql()") { + spark.sessionState.conf.clear() val testKey = "test.key.0" val testVal = "test.val.0" val nonexistentKey = "nonexistent" // "set" itself returns all config variables currently specified in SQLConf. - TestSQLContext.overrideConfs.foreach { case (k, _) => - assert(sql("SET").where(s"key ='$k'").collect().head.get(1) === - TestSQLContext.overrideConfs(k)) + assert(sql("SET").collect().size === TestSQLContext.overrideConfs.size) + sql("SET").collect().foreach { row => + val key = row.getString(0) + val value = row.getString(1) + assert(TestSQLContext.overrideConfs.contains(key), + s"$key should exist in SQLConf.") + assert(TestSQLContext.overrideConfs(key) === value, + s"The value of $key should be ${TestSQLContext.overrideConfs(key)} instead of $value.") } - - val originalConfs = sql("SET").collect() + val overrideConfs = sql("SET").collect() // "set key=val" sql(s"SET $testKey=$testVal") checkAnswer( sql("SET"), - originalConfs ++ Seq(Row(testKey, testVal)) + overrideConfs ++ Seq(Row(testKey, testVal)) ) sql(s"SET ${testKey + testKey}=${testVal + testVal}") checkAnswer( sql("set"), - originalConfs ++ Seq(Row(testKey, testVal), Row(testKey + testKey, testVal + testVal)) + overrideConfs ++ Seq(Row(testKey, testVal), Row(testKey + testKey, testVal + testVal)) ) // "set key" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index c244e3bf7621..59f327075641 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -45,7 +45,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { // Clear the conf. spark.sessionState.conf.clear() // After clear, only overrideConfs used by unit test should be in the SQLConf. - assert((spark.conf.getAll -- TestSQLContext.overrideConfs.keys).size < spark.conf.getAll.size) + assert(spark.conf.getAll === TestSQLContext.overrideConfs) spark.conf.set(testKey, testVal) assert(spark.conf.get(testKey) === testVal) @@ -233,8 +233,8 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { test("default value of WAREHOUSE_PATH") { // JVM adds a trailing slash if the directory exists and leaves it as-is, if it doesn't // In our comparison, strip trailing slash off of both sides, to account for such cases - assert(new Path(Utils.resolveURI("spark-warehouse")).toString.stripSuffix("/") === - spark.sessionState.conf.warehousePath.stripSuffix("/")) + assert(new Path(Utils.resolveURI("spark-warehouse")).toString.stripSuffix("/") === spark + .sessionState.conf.warehousePath.stripSuffix("/")) } test("static SQL conf comes from SparkConf") { From 8363a14994b3b11c8532e8896ca7b7d9b2d28ed0 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 25 Mar 2020 19:07:30 +0800 Subject: [PATCH 08/10] unnecessary --- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 15f53a203d68..de0f7801a39a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1031,12 +1031,14 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark sql("SET").collect().foreach { row => val key = row.getString(0) val value = row.getString(1) - assert(TestSQLContext.overrideConfs.contains(key), + assert( + TestSQLContext.overrideConfs.contains(key), s"$key should exist in SQLConf.") - assert(TestSQLContext.overrideConfs(key) === value, + assert( + TestSQLContext.overrideConfs(key) === value, s"The value of $key should be ${TestSQLContext.overrideConfs(key)} instead of $value.") } - val overrideConfs = sql("SET").collect() + val overrideConfs = sql("SET").collect() // "set key=val" sql(s"SET $testKey=$testVal") From 45c926e478b05deca7232fd9305b5ed1bc622441 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 25 Mar 2020 19:56:23 +0800 Subject: [PATCH 09/10] address comments --- docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 2 +- .../test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index 085f21b638bd..5ebc7b97ef64 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -30,7 +30,7 @@ RESET ### Examples {% highlight sql %} --- Reset all the properties specific to the current session to their default values. +-- Reset any runtime configurations specific to the current session which were set via the SET command to their default values. RESET; {% endhighlight %} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 59f327075641..96f7788c886f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -115,7 +115,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } } - test("reset - static and spark conf") { + test("reset will not change static sql configs and spark core configs") { val conf = spark.sparkContext.getConf.getAll.toMap val appName = conf.get("spark.app.name") val driverHost = conf.get("spark.driver.host") From d374c57903057ac5914ab05597c5bf048ec97b75 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 25 Mar 2020 21:18:29 +0800 Subject: [PATCH 10/10] nit --- .../test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 96f7788c886f..e669794056ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -123,7 +123,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { val warehouseDir = conf.get("spark.sql.warehouse.dir") // ensure the conf here is not default value, and will not be reset to default value later assert(warehouseDir.get.contains(this.getClass.getCanonicalName)) - sql(s"reset") + sql("RESET") assert(conf.get("spark.app.name") === appName) assert(conf.get("spark.driver.host") === driverHost) assert(conf.get("spark.master") === master)