You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2020/03/26 07:14:20 UTC
[spark] branch branch-3.0 updated: [SPARK-31234][SQL] ResetCommand
should reset config to sc.conf only
This is an automated email from the ASF dual-hosted git repository.
wenchen pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.0 by this push:
new 557623b [SPARK-31234][SQL] ResetCommand should reset config to sc.conf only
557623b is described below
commit 557623b5a1ebd50b0f604843628b9d0ed8ff19ef
Author: Kent Yao <ya...@hotmail.com>
AuthorDate: Thu Mar 26 15:03:16 2020 +0800
[SPARK-31234][SQL] ResetCommand should reset config to sc.conf only
### What changes were proposed in this pull request?
Currently, ResetCommand clear all configurations, including sql configs, static sql configs and spark context level configs.
for example:
```sql
spark-sql> set xyz=abc;
xyz abc
spark-sql> set;
spark.app.id local-1585055396930
spark.app.name SparkSQL::10.242.189.214
spark.driver.host 10.242.189.214
spark.driver.port 65094
spark.executor.id driver
spark.jars
spark.master local[*]
spark.sql.catalogImplementation hive
spark.sql.hive.version 1.2.1
spark.submit.deployMode client
xyz abc
spark-sql> reset;
spark-sql> set;
spark-sql> set spark.sql.hive.version;
spark.sql.hive.version 1.2.1
spark-sql> set spark.app.id;
spark.app.id <undefined>
```
In this PR, we restore spark confs to RuntimeConfig after it is cleared
### Why are the changes needed?
reset command overkills configs which are static.
### Does this PR introduce any user-facing change?
yes, the ResetCommand do not change static configs now
### How was this patch tested?
add ut
Closes #28003 from yaooqinn/SPARK-31234.
Authored-by: Kent Yao <ya...@hotmail.com>
Signed-off-by: Wenchen Fan <we...@databricks.com>
(cherry picked from commit 44bd36ad7b315f4c7592cdc1edf04356fcd23645)
Signed-off-by: Wenchen Fan <we...@databricks.com>
---
docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 4 ++--
.../apache/spark/sql/execution/command/SetCommand.scala | 9 +++++++--
.../org/apache/spark/sql/internal/SQLConfSuite.scala | 15 +++++++++++++++
3 files changed, 24 insertions(+), 4 deletions(-)
diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md
index 8ee6151..5ebc7b9 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 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 %}
@@ -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/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 39b08e2..a12b261 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
@@ -157,7 +157,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;
* }}}
@@ -165,7 +166,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 cde2aa7..c2d8493d 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,6 +116,21 @@ class SQLConfSuite extends QueryTest with SharedSparkSession {
}
}
+ 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")
+ 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 later
+ assert(warehouseDir.get.contains(this.getClass.getCanonicalName))
+ sql("RESET")
+ 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") {
spark.sessionState.conf.clear()
val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL)
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org