Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-31234][SQL][2.4] ResetCommand should not affect static SQL Configuration #28262

Closed
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
6 changes: 5 additions & 1 deletion docs/sql-migration-guide-upgrade.md
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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]
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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]].
Expand Down Expand Up @@ -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")
Copy link
Member Author

Choose a reason for hiding this comment

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

GLOBAL_TEMP_DATABASE is a typical example. The value is being used at the runtime. See the code

val db = sparkSession.sessionState.conf.getConf(StaticSQLConf.GLOBAL_TEMP_DATABASE)

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down