This is an automated email from the ASF dual-hosted git repository.
dongjoon pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-2.4 by this push:
new 1af3b48 [SPARK-31234][SQL][2.4] ResetCommand should not affect static
SQL Configuration
1af3b48 is described below
commit 1af3b48486e4b60f226ed44dcb23a7e6379c99a6
Author: gatorsmile <[email protected]>
AuthorDate: Sun Apr 19 16:24:56 2020 -0700
[SPARK-31234][SQL][2.4] ResetCommand should not affect static SQL
Configuration
### What changes were proposed in this pull request?
This PR is to backport the fix of
https://github.com/apache/spark/pull/28003, add a migration guide, update the
PR description and add an end-to-end test case.
Before this PR, the SQL `RESET` command will reset the values of static SQL
configuration to the default and remove the cached values of Spark Context
Configurations in the current session. This PR fixes the bugs. After this PR,
the `RESET` command follows its definition and only updates the runtime SQL
configuration values to the default.
### Why are the changes needed?
When we introduced the feature of Static SQL Configuration, we did not
update the implementation of SQL `RESET` command.
The static SQL configuration should not be changed by any command at
runtime. However, the `RESET` command resets the values to the default. We
should fix them.
### Does this PR introduce any user-facing change?
Before Spark 2.4.6, the `RESET` command resets both the runtime and static
SQL configuration values to the default. It also removes the cached values of
Spark Context Configurations in the current session, although these
configuration values are for displaying/querying only.
### How was this patch tested?
Added an end-to-end test and a unit test
Closes #28262 from gatorsmile/spark-31234followup2.4.
Authored-by: gatorsmile <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
---
docs/sql-migration-guide-upgrade.md | 6 +++++-
.../apache/spark/sql/execution/command/SetCommand.scala | 6 +++++-
.../org/apache/spark/sql/SparkSessionBuilderSuite.scala | 16 ++++++++++++++++
.../org/apache/spark/sql/internal/SQLConfSuite.scala | 15 +++++++++++++++
4 files changed, 41 insertions(+), 2 deletions(-)
diff --git a/docs/sql-migration-guide-upgrade.md
b/docs/sql-migration-guide-upgrade.md
index 8170637..09cf834 100644
--- a/docs/sql-migration-guide-upgrade.md
+++ b/docs/sql-migration-guide-upgrade.md
@@ -7,7 +7,11 @@ displayTitle: Spark SQL Upgrading Guide
* Table of contents
{:toc}
-## Upgrading from Spark SQL 2.4 to 2.4.5
+## Upgrading from Spark SQL 2.4.5 to 2.4.6
+
+ - In Spark 2.4.6, the `RESET` command does not reset the static SQL
configuration values to the default. It only clears the runtime SQL
configuration values.
+
+## Upgrading from Spark SQL 2.4.4 to 2.4.5
- Starting from 2.4.5, SQL configurations are effective also when a Dataset
is converted to an RDD and its
plan is executed due to action on the derived RDD. The previous behavior
can be restored setting
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 3c900be..435cdf2 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
@@ -164,7 +164,11 @@ object SetCommand {
case object ResetCommand extends RunnableCommand with Logging {
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/SparkSessionBuilderSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala
index 44bf862..a57f09e 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala
@@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfterEach
import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE
/**
* Test cases for the builder pattern of [[SparkSession]].
@@ -151,4 +152,19 @@ class SparkSessionBuilderSuite extends SparkFunSuite with
BeforeAndAfterEach {
session.sparkContext.hadoopConfiguration.unset(mySpecialKey)
}
}
+
+ test("SPARK-31234: RESET command will not change static sql configs and " +
+ "spark context conf values in SessionState") {
+ val session = SparkSession.builder()
+ .master("local")
+ .config(GLOBAL_TEMP_DATABASE.key, value = "globalTempDB-SPARK-31234")
+ .config("spark.app.name", "test-app-SPARK-31234")
+ .getOrCreate()
+
+ assert(session.sessionState.conf.getConfString("spark.app.name") ===
"test-app-SPARK-31234")
+ assert(session.sessionState.conf.getConf(GLOBAL_TEMP_DATABASE) ===
"globalTempDB-SPARK-31234")
+ session.sql("RESET")
+ assert(session.sessionState.conf.getConfString("spark.app.name") ===
"test-app-SPARK-31234")
+ assert(session.sessionState.conf.getConf(GLOBAL_TEMP_DATABASE) ===
"globalTempDB-SPARK-31234")
+ }
}
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 c9a6975..e965ad4 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
@@ -114,6 +114,21 @@ class SQLConfSuite extends QueryTest with SharedSQLContext
{
}
}
+ test("SPARK-31234: 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: [email protected]
For additional commands, e-mail: [email protected]