yaooqinn opened a new pull request #28316:
URL: https://github.com/apache/spark/pull/28316


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: 
https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: 
https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., 
'[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a 
faster review.
     7. If you want to add a new configuration, please read the guideline first 
for naming configurations in
        
'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### 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.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?
   <!--
   If yes, please clarify the previous behavior and the change this PR proposes 
- provide the console output, description and/or an example to show the 
behavior difference if possible.
   If no, write 'No'.
   -->
   Yes, static SQL configurations with SparkSession.builder.config do not 
propagate to any existing or new SparkSession instances.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some 
test cases that check the changes thoroughly including negative and positive 
cases if possible.
   If it was tested in a way different from regular unit tests, please clarify 
how you tested step by step, ideally copy and paste-able, so that other 
reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why 
it was difficult to add.
   -->
   new ut.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to