Skip to content

Conversation

@yaooqinn
Copy link
Member

What changes were proposed in this pull request?

SparkSessionBuilder shoud not propagate static sql configurations to the existing active/default SparkSession
This seems a long-standing bug.

scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+--------------------+
|                 key|               value|
+--------------------+--------------------+
|spark.sql.warehou...|file:/Users/kenty...|
+--------------------+--------------------+


scala> spark.sql("set spark.sql.warehouse.dir=2");
org.apache.spark.sql.AnalysisException: Cannot modify the value of a static config: spark.sql.warehouse.dir;
  at org.apache.spark.sql.RuntimeConfig.requireNonStaticConf(RuntimeConfig.scala:154)
  at org.apache.spark.sql.RuntimeConfig.set(RuntimeConfig.scala:42)
  at org.apache.spark.sql.execution.command.SetCommand.$anonfun$x$7$6(SetCommand.scala:100)
  at org.apache.spark.sql.execution.command.SetCommand.run(SetCommand.scala:156)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
  at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
  at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:607)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:602)
  ... 47 elided

scala> import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.SparkSession

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").get
getClass   getOrCreate

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").getOrCreate
20/04/23 23:49:13 WARN SparkSession$Builder: Using an existing SparkSession; some configuration may not take effect.
res7: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession@6403d574

scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+-----+
|                 key|value|
+--------------------+-----+
|spark.sql.warehou...|  xyz|
+--------------------+-----+


scala>
OptionsAttachments

Why are the changes needed?

bugfix as shown in the previous section

Does this PR introduce any user-facing change?

Yes, static SQL configurations with SparkSession.builder.config do not propagate to any existing or new SparkSession instances.

How was this patch tested?

new ut.

@yaooqinn
Copy link
Member Author

cc: @cloud-fan @dongjoon-hyun @maropu @HyukjinKwon many thanks.

@SparkQA
Copy link

SparkQA commented Apr 23, 2020

Test build #121688 has finished for PR 28316 at commit 92bc41b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

var session = activeThreadSession.get()
if ((session ne null) && !session.sparkContext.isStopped) {
options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) }
for ((k, v) <- options if !SQLConf.staticConfKeys.contains(k)) {
Copy link
Member

Choose a reason for hiding this comment

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

Could we have clearer warning message than configuration may not take effect if options has any static config?

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

Copy link
Member Author

Choose a reason for hiding this comment

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

+1 too. we can warn static SQL separately. But do we still need a general warning messages for core configurations?
e.g.

for ((k, v) <- options) {
        if (SQLConf.staticConfKeys.contains(k)) {
          logWarning(s"Using an existing SparkSession, the static configuration $k is ignored.")
        } else {
          session.sessionState.conf.setConfString(k, v)
        }
      }
      if ((options -- SQLConf.staticConfKeys.asScala).nonEmpty) {
        logWarning("Using an existing SparkSession; some spark core configurations may not take" +
          " effect.")
      }

Copy link
Member Author

Choose a reason for hiding this comment

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

+1 too. we can warn static SQL separately. But do we still need a general warning messages for core configurations?
e.g.

for ((k, v) <- options) {
        if (SQLConf.staticConfKeys.contains(k)) {
          logWarning(s"Using an existing SparkSession, the static configuration $k is ignored.")
        } else {
          session.sessionState.conf.setConfString(k, v)
        }
      }
      if ((options -- SQLConf.staticConfKeys.asScala).nonEmpty) {
        logWarning("Using an existing SparkSession; some spark core configurations may not take" +
          " effect.")
      }

assert(!session.conf.get(WAREHOUSE_PATH).contains("SPARK-31532-db"))
assert(session.conf.get(WAREHOUSE_PATH) === session2.conf.get(WAREHOUSE_PATH))
assert(session2.conf.get(GLOBAL_TEMP_DATABASE) === "globaltempdb-spark-31532")

Copy link
Member

Choose a reason for hiding this comment

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

nit: remove this blank

@maropu
Copy link
Member

maropu commented Apr 24, 2020

Nice catch! Looks fine.


test("SPARK-31532: should not propagate static sql configs to the existing" +
" active/default SparkSession") {
val session = SparkSession.builder()
Copy link
Member

Choose a reason for hiding this comment

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

Just in case, could we add tests for the case where we should set static configs at SparkSession (In case that no active/default SparkSession exists)? Or, we already have such a test?

Copy link
Member Author

Choose a reason for hiding this comment

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

I add a new test for such a case, that is, if SparkContext instance exists but no SparkSession exists, then the static configs remain changeable before SparkSession is finally created.

}

private def applyModifiableSettings(session: SparkSession): Unit = {
for ((k, v) <- options) {
Copy link
Contributor

Choose a reason for hiding this comment

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

how about

val (staticConfs, otherConfs) = options.partition(kv => SQLConf.staticConfKeys.contains(kv._1))
if (staticConfs.nonEmpty) warn...
if (otherConfs.nonEmpty) warn...

@SparkQA
Copy link

SparkQA commented Apr 24, 2020

Test build #121714 has finished for PR 28316 at commit 0c6a270.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 24, 2020

Test build #121712 has finished for PR 28316 at commit ad91ccb.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@maropu
Copy link
Member

maropu commented Apr 24, 2020

retest this please

@SparkQA
Copy link

SparkQA commented Apr 24, 2020

Test build #121730 has finished for PR 28316 at commit 0c6a270.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@cloud-fan
Copy link
Contributor

retest this please

@SparkQA
Copy link

SparkQA commented Apr 24, 2020

Test build #121761 has finished for PR 28316 at commit 0c6a270.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@maropu maropu closed this in 8424f55 Apr 24, 2020
maropu pushed a commit that referenced this pull request Apr 24, 2020
… the existing active or default SparkSession

### What changes were proposed in this pull request?

SparkSessionBuilder shoud not propagate static sql configurations to the existing active/default SparkSession
This seems a long-standing bug.

```scala
scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+--------------------+
|                 key|               value|
+--------------------+--------------------+
|spark.sql.warehou...|file:/Users/kenty...|
+--------------------+--------------------+

scala> spark.sql("set spark.sql.warehouse.dir=2");
org.apache.spark.sql.AnalysisException: Cannot modify the value of a static config: spark.sql.warehouse.dir;
  at org.apache.spark.sql.RuntimeConfig.requireNonStaticConf(RuntimeConfig.scala:154)
  at org.apache.spark.sql.RuntimeConfig.set(RuntimeConfig.scala:42)
  at org.apache.spark.sql.execution.command.SetCommand.$anonfun$x$7$6(SetCommand.scala:100)
  at org.apache.spark.sql.execution.command.SetCommand.run(SetCommand.scala:156)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
  at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
  at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:607)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:602)
  ... 47 elided

scala> import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.SparkSession

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").get
getClass   getOrCreate

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").getOrCreate
20/04/23 23:49:13 WARN SparkSession$Builder: Using an existing SparkSession; some configuration may not take effect.
res7: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession6403d574

scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+-----+
|                 key|value|
+--------------------+-----+
|spark.sql.warehou...|  xyz|
+--------------------+-----+

scala>
OptionsAttachments
```

### Why are the changes needed?
bugfix as shown in the previous section

### Does this PR introduce any user-facing change?

Yes, static SQL configurations with SparkSession.builder.config do not propagate to any existing or new SparkSession instances.

### How was this patch tested?

new ut.

Closes #28316 from yaooqinn/SPARK-31532.

Authored-by: Kent Yao <[email protected]>
Signed-off-by: Takeshi Yamamuro <[email protected]>
(cherry picked from commit 8424f55)
Signed-off-by: Takeshi Yamamuro <[email protected]>
maropu pushed a commit that referenced this pull request Apr 24, 2020
… the existing active or default SparkSession

### What changes were proposed in this pull request?

SparkSessionBuilder shoud not propagate static sql configurations to the existing active/default SparkSession
This seems a long-standing bug.

```scala
scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+--------------------+
|                 key|               value|
+--------------------+--------------------+
|spark.sql.warehou...|file:/Users/kenty...|
+--------------------+--------------------+

scala> spark.sql("set spark.sql.warehouse.dir=2");
org.apache.spark.sql.AnalysisException: Cannot modify the value of a static config: spark.sql.warehouse.dir;
  at org.apache.spark.sql.RuntimeConfig.requireNonStaticConf(RuntimeConfig.scala:154)
  at org.apache.spark.sql.RuntimeConfig.set(RuntimeConfig.scala:42)
  at org.apache.spark.sql.execution.command.SetCommand.$anonfun$x$7$6(SetCommand.scala:100)
  at org.apache.spark.sql.execution.command.SetCommand.run(SetCommand.scala:156)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
  at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
  at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:607)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:602)
  ... 47 elided

scala> import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.SparkSession

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").get
getClass   getOrCreate

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").getOrCreate
20/04/23 23:49:13 WARN SparkSession$Builder: Using an existing SparkSession; some configuration may not take effect.
res7: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession6403d574

scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+-----+
|                 key|value|
+--------------------+-----+
|spark.sql.warehou...|  xyz|
+--------------------+-----+

scala>
OptionsAttachments
```

### Why are the changes needed?
bugfix as shown in the previous section

### Does this PR introduce any user-facing change?

Yes, static SQL configurations with SparkSession.builder.config do not propagate to any existing or new SparkSession instances.

### How was this patch tested?

new ut.

Closes #28316 from yaooqinn/SPARK-31532.

Authored-by: Kent Yao <[email protected]>
Signed-off-by: Takeshi Yamamuro <[email protected]>
(cherry picked from commit 8424f55)
Signed-off-by: Takeshi Yamamuro <[email protected]>
@maropu
Copy link
Member

maropu commented Apr 24, 2020

Thanks, @yaooqinn @cloud-fan ! Merged to master/3.0/2.4 according to SPARK-31532.

@maropu
Copy link
Member

maropu commented Apr 24, 2020

FYI: @gatorsmile @dongjoon-hyun

@dongjoon-hyun
Copy link
Member

Thank you for pinging me, @maropu .
BTW, unfortunately, this seems to break branch-2.4.

org.apache.spark.sql.SparkSessionBuilderSuite.SPARK-31532: should not propagate static sql configs to the existing active/default SparkSession
org.scalatest.exceptions.TestFailedException: "global[TempDB-SPARK]-31532" did not equal "global[tempdb-spark]-31532"

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Apr 25, 2020

We need SPARK-28179 (#24979) to fix the failure, but it will be a behavior change in branch-2.4. Or, we can fix the test case to lowercase. Or, simply revert this from branch-2.4.

cc @gatorsmile , @HyukjinKwon , @srowen

@maropu
Copy link
Member

maropu commented Apr 25, 2020

Oops, I forgot the Xiao's comment #28262 (comment) ...
How about just fixing the test suite? This is a bugfix, so I personally think branch-2.4 needs this fix.

@dongjoon-hyun
Copy link
Member

In branch-2.4, the following is the problem.

  val GLOBAL_TEMP_DATABASE = buildStaticConf("spark.sql.globalTempDatabase")
    .internal()
    .stringConf
    .createWithDefault("global_temp")

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Apr 25, 2020

I took a look. It looks okay to change the conf itself.

@dongjoon-hyun
Copy link
Member

Could you make a follow-up PR with the following?

    // System preserved database should not exists in metastore. However it's hard to guarantee it
    // for every session, because case-sensitivity differs. Here we always lowercase it to make our
    // life easier.
    .transform(_.toLowerCase(Locale.ROOT))

@maropu
Copy link
Member

maropu commented Apr 25, 2020

Yea, ok. I'll do now.

@dongjoon-hyun
Copy link
Member

Technically, the difference is the stored value in conf. The generated table is lower case always due to val globalTempDB = sparkContext.conf.get(GLOBAL_TEMP_DATABASE).toLowerCase(Locale.ROOT).

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Apr 25, 2020

Ping the related people including me on the follow-PR. Thanks!

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Apr 25, 2020

Ah.. There is an old test case using case-sensitive test.

@dongjoon-hyun
Copy link
Member

Please update them together. And, let's see the test result.

sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala:    assert(session.sessionState.conf.getConf(GLOBAL_TEMP_DATABASE) === "globalTempDB-SPARK-31234")
sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala:    assert(session.sessionState.conf.getConf(GLOBAL_TEMP_DATABASE) === "globalTempDB-SPARK-31234")

dongjoon-hyun pushed a commit that referenced this pull request Apr 25, 2020
…BAL_TEMP_DATABASE config in SparkSessionBuilderSuite

### What changes were proposed in this pull request?

This PR intends to fix test code for using lowercases for the `GLOBAL_TEMP_DATABASE` config in `SparkSessionBuilderSuite`. The handling of the config is different between branch-3.0+ and branch-2.4. In branch-3.0+, Spark always lowercases a value in the config, so I think we had better always use lowercases for it in the test.

This comes from the dongjoon-hyun
 comment: #28316 (comment)

### Why are the changes needed?

To fix the test failure in branch-2.4.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Fixed the test.

Closes #28339 from maropu/SPARK-31532.

Authored-by: Takeshi Yamamuro <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
scunniff pushed a commit to scunniff/nomad-spark that referenced this pull request Nov 10, 2020
… the existing active or default SparkSession

### What changes were proposed in this pull request?

SparkSessionBuilder shoud not propagate static sql configurations to the existing active/default SparkSession
This seems a long-standing bug.

```scala
scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+--------------------+
|                 key|               value|
+--------------------+--------------------+
|spark.sql.warehou...|file:/Users/kenty...|
+--------------------+--------------------+

scala> spark.sql("set spark.sql.warehouse.dir=2");
org.apache.spark.sql.AnalysisException: Cannot modify the value of a static config: spark.sql.warehouse.dir;
  at org.apache.spark.sql.RuntimeConfig.requireNonStaticConf(RuntimeConfig.scala:154)
  at org.apache.spark.sql.RuntimeConfig.set(RuntimeConfig.scala:42)
  at org.apache.spark.sql.execution.command.SetCommand.$anonfun$x$7$6(SetCommand.scala:100)
  at org.apache.spark.sql.execution.command.SetCommand.run(SetCommand.scala:156)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
  at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
  at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
  at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
  at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
  at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
  at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
  at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
  at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:607)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:602)
  ... 47 elided

scala> import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.SparkSession

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").get
getClass   getOrCreate

scala> SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").getOrCreate
20/04/23 23:49:13 WARN SparkSession$Builder: Using an existing SparkSession; some configuration may not take effect.
res7: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession6403d574

scala> spark.sql("set spark.sql.warehouse.dir").show
+--------------------+-----+
|                 key|value|
+--------------------+-----+
|spark.sql.warehou...|  xyz|
+--------------------+-----+

scala>
OptionsAttachments
```

### Why are the changes needed?
bugfix as shown in the previous section

### Does this PR introduce any user-facing change?

Yes, static SQL configurations with SparkSession.builder.config do not propagate to any existing or new SparkSession instances.

### How was this patch tested?

new ut.

Closes apache#28316 from yaooqinn/SPARK-31532.

Authored-by: Kent Yao <[email protected]>
Signed-off-by: Takeshi Yamamuro <[email protected]>
(cherry picked from commit 8424f55)
Signed-off-by: Takeshi Yamamuro <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants