Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions docs/sql-ref-syntax-aux-conf-mgmt-reset.md
Original file line number Diff line number Diff line change
Expand Up @@ -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.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need to update the comment below, too?

-- Reset "all the properties" specific to the current session to their default values.
RESET;

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good catch!

### Syntax
{% highlight sql %}
Expand All @@ -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 %}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,15 +162,20 @@ 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;
* }}}
*/
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]
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,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)
Expand Down