Skip to content
This repository was archived by the owner on Oct 23, 2024. It is now read-only.

Commit 557623b

Browse files
yaooqinncloud-fan
authored andcommitted
[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 apache#28003 from yaooqinn/SPARK-31234. Authored-by: Kent Yao <[email protected]> Signed-off-by: Wenchen Fan <[email protected]> (cherry picked from commit 44bd36a) Signed-off-by: Wenchen Fan <[email protected]>
1 parent a36e3c4 commit 557623b

File tree

3 files changed

+24
-4
lines changed

3 files changed

+24
-4
lines changed

docs/sql-ref-syntax-aux-conf-mgmt-reset.md

+2-2
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ license: |
2020
---
2121

2222
### Description
23-
Reset all the properties specific to the current session to their default values. After RESET command, executing SET command will output empty.
23+
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.
2424

2525
### Syntax
2626
{% highlight sql %}
@@ -30,7 +30,7 @@ RESET
3030

3131
### Examples
3232
{% highlight sql %}
33-
-- Reset all the properties specific to the current session to their default values.
33+
-- Reset any runtime configurations specific to the current session which were set via the SET command to their default values.
3434
RESET;
3535
{% endhighlight %}
3636

sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala

+7-2
Original file line numberDiff line numberDiff line change
@@ -157,15 +157,20 @@ object SetCommand {
157157
}
158158

159159
/**
160-
* This command is for resetting SQLConf to the default values. Command that runs
160+
* This command is for resetting SQLConf to the default values. Any configurations that were set
161+
* via [[SetCommand]] will get reset to default value. Command that runs
161162
* {{{
162163
* reset;
163164
* }}}
164165
*/
165166
case object ResetCommand extends RunnableCommand with IgnoreCachedData {
166167

167168
override def run(sparkSession: SparkSession): Seq[Row] = {
168-
sparkSession.sessionState.conf.clear()
169+
val conf = sparkSession.sessionState.conf
170+
conf.clear()
171+
sparkSession.sparkContext.conf.getAll.foreach { case (k, v) =>
172+
conf.setConfString(k, v)
173+
}
169174
Seq.empty[Row]
170175
}
171176
}

sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala

+15
Original file line numberDiff line numberDiff line change
@@ -116,6 +116,21 @@ class SQLConfSuite extends QueryTest with SharedSparkSession {
116116
}
117117
}
118118

119+
test("reset will not change static sql configs and spark core configs") {
120+
val conf = spark.sparkContext.getConf.getAll.toMap
121+
val appName = conf.get("spark.app.name")
122+
val driverHost = conf.get("spark.driver.host")
123+
val master = conf.get("spark.master")
124+
val warehouseDir = conf.get("spark.sql.warehouse.dir")
125+
// ensure the conf here is not default value, and will not be reset to default value later
126+
assert(warehouseDir.get.contains(this.getClass.getCanonicalName))
127+
sql("RESET")
128+
assert(conf.get("spark.app.name") === appName)
129+
assert(conf.get("spark.driver.host") === driverHost)
130+
assert(conf.get("spark.master") === master)
131+
assert(conf.get("spark.sql.warehouse.dir") === warehouseDir)
132+
}
133+
119134
test("reset - public conf") {
120135
spark.sessionState.conf.clear()
121136
val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL)

0 commit comments

Comments
 (0)