From d3596c04b0275b19d6edc0126a77f749b4e9ba70 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 24 Jul 2020 09:13:26 -0700 Subject: [PATCH] [SPARK-32406][SQL] Make RESET syntax support single configuration reset MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR extends the RESET command to support reset SQL configuration one by one. ### Why are the changes needed? Currently, the reset command only supports restore all of the runtime configurations to their defaults. In most cases, users do not want this, but just want to restore one or a small group of settings. The SET command can work as a workaround for this, but you have to keep the defaults in your mind or by temp variables, which turns out not very convenient to use. Hive supports this: https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients#HiveServer2Clients-BeelineExample reset | Resets the value of a particular configuration variable (key) to the default value.Note: If you misspell the variable name, Beeline will not show an error. -- | -- PostgreSQL supports this too https://www.postgresql.org/docs/9.1/sql-reset.html ### Does this PR introduce _any_ user-facing change? yes, reset can restore one configuration now ### How was this patch tested? add new unit tests. Closes #29202 from yaooqinn/SPARK-32406. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- docs/sql-ref-syntax-aux-conf-mgmt-reset.md | 19 +++++++++++++++++-- .../spark/sql/catalyst/parser/SqlBase.g4 | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 3 ++- .../sql/execution/command/SetCommand.scala | 14 ++++++++++---- .../spark/sql/internal/SQLConfSuite.scala | 15 +++++++++++++++ 5 files changed, 45 insertions(+), 8 deletions(-) diff --git a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md index 4caf57a232..68df0913b2 100644 --- a/docs/sql-ref-syntax-aux-conf-mgmt-reset.md +++ b/docs/sql-ref-syntax-aux-conf-mgmt-reset.md @@ -21,19 +21,34 @@ license: | ### Description -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. +The RESET command resets 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 ```sql -RESET +RESET; + +RESET configuration_key; ``` +### Parameters + +* **(none)** + + 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. + +* **configuration_key** + + Restore the value of the `configuration_key` to the default value. If the default value is undefined, the `configuration_key` will be removed. + ### Examples ```sql -- Reset any runtime configurations specific to the current session which were set via the SET command to their default values. RESET; + +-- If you start your application with --conf spark.foo=bar and set spark.foo=foobar in runtime, the example below will restore it to 'bar'. If spark.foo is not specified during starting, the example bellow will remove this config from the SQLConf. It will ignore nonexistent keys. +RESET spark.abc; ``` ### Related Statements diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index b9c46616ca..4631165981 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -245,7 +245,7 @@ statement | SET TIME ZONE timezone=(STRING | LOCAL) #setTimeZone | SET TIME ZONE .*? #setTimeZone | SET .*? #setConfiguration - | RESET #resetConfiguration + | RESET .*? #resetConfiguration | unsupportedHiveNativeCommands .*? #failNativeCommand ; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 44d5285e56..012ae0a760 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -85,11 +85,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * Example SQL : * {{{ * RESET; + * RESET spark.sql.session.timeZone; * }}} */ override def visitResetConfiguration( ctx: ResetConfigurationContext): LogicalPlan = withOrigin(ctx) { - ResetCommand + ResetCommand(Option(remainder(ctx.RESET().getSymbol).trim).filter(_.nonEmpty)) } /** 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 3dc1d52697..24c43ee407 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 @@ -166,15 +166,21 @@ object SetCommand { * via [[SetCommand]] will get reset to default value. Command that runs * {{{ * reset; + * reset spark.sql.session.timeZone; * }}} */ -case object ResetCommand extends RunnableCommand with IgnoreCachedData { +case class ResetCommand(config: Option[String]) extends RunnableCommand with IgnoreCachedData { override def run(sparkSession: SparkSession): Seq[Row] = { val conf = sparkSession.sessionState.conf - conf.clear() - sparkSession.sparkContext.conf.getAll.foreach { case (k, v) => - conf.setConfString(k, v) + val defaults = sparkSession.sparkContext.conf + config match { + case Some(key) => + conf.unsetConf(key) + defaults.getOption(key).foreach(conf.setConfString(key, _)) + case None => + conf.clear() + defaults.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 36cb5d94a0..0ecc5ee04c 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 @@ -181,6 +181,21 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } } + test("SPARK-32406: reset - single configuration") { + val appId = spark.sparkContext.getConf.getAppId + sql("RESET spark.app.id") + assert(spark.conf.get("spark.app.id") === appId, "Should not change spark core ones") + sql("SET spark.abc=xyz") + assert(spark.conf.get("spark.abc") === "xyz") + sql("RESET spark.abc") + intercept[NoSuchElementException](spark.conf.get("spark.abc")) + sql("RESET spark.abc") // ignore nonexistent keys + val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) + sql(s"SET ${SQLConf.GROUP_BY_ORDINAL.key}=false") + sql(s"RESET ${SQLConf.GROUP_BY_ORDINAL.key}") + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === original) + } + test("invalid conf value") { spark.sessionState.conf.clear() val e = intercept[IllegalArgumentException] {