diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 817063770f68..09cf834939ba 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 3c900be839aa..435cdf25a11f 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 44bf8624a6bc..a57f09e677fd 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 c9a6975da6be..e965ad4c75e3 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)