You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2020/07/31 13:13:55 UTC
[spark] branch master updated: [SPARK-32406][SQL][FOLLOWUP] Make
RESET fail against static and core configs
This is an automated email from the ASF dual-hosted git repository.
gurwls223 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new f480040 [SPARK-32406][SQL][FOLLOWUP] Make RESET fail against static and core configs
f480040 is described below
commit f4800406a455b33afa7b1d62d10f236da4cd1f83
Author: Kent Yao <ya...@hotmail.com>
AuthorDate: Fri Jul 31 22:11:05 2020 +0900
[SPARK-32406][SQL][FOLLOWUP] Make RESET fail against static and core configs
### What changes were proposed in this pull request?
This followup addresses comments from https://github.com/apache/spark/pull/29202#discussion_r462054784
1. make RESET static SQL configs/spark core configs fail as same as the SET command. Not that, for core ones, they have to be pre-registered, otherwise, they are still able to be SET/RESET
2. add test cases for configurations w/ optional default values
### Why are the changes needed?
behavior change with suggestions from PMCs
### Does this PR introduce _any_ user-facing change?
Yes, RESET will fail after this PR, before it just does nothing because the static ones are static.
### How was this patch tested?
add more tests.
Closes #29297 from yaooqinn/SPARK-32406-F.
Authored-by: Kent Yao <ya...@hotmail.com>
Signed-off-by: HyukjinKwon <gu...@apache.org>
---
.../spark/sql/execution/command/SetCommand.scala | 11 +++++----
.../apache/spark/sql/internal/SQLConfSuite.scala | 27 ++++++++++++++++++++++
2 files changed, 33 insertions(+), 5 deletions(-)
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 24c43ee..fd89e36 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
@@ -172,15 +172,16 @@ object SetCommand {
case class ResetCommand(config: Option[String]) extends RunnableCommand with IgnoreCachedData {
override def run(sparkSession: SparkSession): Seq[Row] = {
- val conf = sparkSession.sessionState.conf
val defaults = sparkSession.sparkContext.conf
config match {
case Some(key) =>
- conf.unsetConf(key)
- defaults.getOption(key).foreach(conf.setConfString(key, _))
+ sparkSession.conf.unset(key)
+ defaults.getOption(key).foreach(sparkSession.conf.set(key, _))
case None =>
- conf.clear()
- defaults.getAll.foreach { case (k, v) => conf.setConfString(k, v) }
+ sparkSession.sessionState.conf.clear()
+ defaults.getAll.foreach { case (k, v) =>
+ sparkSession.sessionState.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 0ecc5ee..565a835 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
@@ -142,9 +142,12 @@ class SQLConfSuite extends QueryTest with SharedSparkSession {
sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false")
assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false)
assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1)
+ assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES).isEmpty)
sql(s"reset")
assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL))
assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0)
+ assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES) ===
+ Some("org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation"))
} finally {
sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original")
}
@@ -182,18 +185,42 @@ class SQLConfSuite extends QueryTest with SharedSparkSession {
}
test("SPARK-32406: reset - single configuration") {
+ spark.sessionState.conf.clear()
+ // spark core conf w/o entry registered
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")
+ // spark core conf w/ entry registered
+ val e1 = intercept[AnalysisException](sql("RESET spark.executor.cores"))
+ assert(e1.getMessage === "Cannot modify the value of a Spark config: spark.executor.cores;")
+
+ // user defined settings
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
+
+ // runtime sql configs
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)
+
+ // runtime sql configs with optional defaults
+ assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES).isEmpty)
+ sql(s"RESET ${SQLConf.OPTIMIZER_EXCLUDED_RULES.key}")
+ assert(spark.conf.get(SQLConf.OPTIMIZER_EXCLUDED_RULES) ===
+ Some("org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation"))
+ sql(s"SET ${SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_RULES.key}=abc")
+ sql(s"RESET ${SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_RULES.key}")
+ assert(spark.conf.get(SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_RULES).isEmpty)
+
+ // static sql configs
+ val e2 = intercept[AnalysisException](sql(s"RESET ${StaticSQLConf.WAREHOUSE_PATH.key}"))
+ assert(e2.getMessage ===
+ s"Cannot modify the value of a static config: ${StaticSQLConf.WAREHOUSE_PATH.key};")
+
}
test("invalid conf value") {
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org