HyukjinKwon opened a new pull request #35001:
URL: https://github.com/apache/spark/pull/35001


   ### What changes were proposed in this pull request?
   
   This PR proposes to show ignored configurations and hide the warnings for 
configurations that are already set when invoking 
`SparkSession.builder.getOrCreate`.
   
   ### Why are the changes needed?
   
   Currently, `SparkSession.builder.getOrCreate()` is too noisy even when 
duplicate configurations are set. Users cannot easily tell which configurations 
are to fix. See the example below:
   
   ```bash
   ./bin/spark-shell --conf spark.abc=abc
   ```
   
   ```scala
   import org.apache.spark.sql.SparkSession
   spark.sparkContext.setLogLevel("DEBUG")
   SparkSession.builder.config("spark.abc", "abc").getOrCreate
   ```
   
   ```
   21:04:01.670 [main] WARN  org.apache.spark.sql.SparkSession - Using an 
existing SparkSession; some spark core configurations may not take effect.
   ```
   
   It is straitforward when there are few configurations but it is difficult 
for users to figure out when there are too many configurations especially when 
these configurations are defined in a property file like {{spark-default.conf}} 
maintained separately by system admins in production.
   
   See also https://github.com/apache/spark/pull/34757#discussion_r769248275.
   
   
   ### Does this PR introduce _any_ user-facing change?
   
   Yes.
   
   1. Show ignored configuration in debug level logs:
   
       ```bash
       ./bin/spark-shell --conf spark.abc=abc
       ```
   
       ```scala
       import org.apache.spark.sql.SparkSession
       spark.sparkContext.setLogLevel("DEBUG")
       SparkSession.builder
         .config("spark.sql.warehouse.dir", "2")
         .config("spark.abc", "abcb")
         .config("spark.abcd", "abcb4")
         .getOrCreate
       ```
   
       **Before:**
   
       ```
       21:13:28.360 [main] WARN  org.apache.spark.sql.SparkSession - Using an 
existing SparkSession; the static sql configurations will not take effect.
       21:13:28.360 [main] WARN  org.apache.spark.sql.SparkSession - Using an 
existing SparkSession; some spark core configurations may not take effect.
       ```
   
       **After**:
   
       ```
       20:34:30.619 [main] WARN  org.apache.spark.sql.SparkSession - Using an 
existing Spark session; only runtime SQL configurations will take effect.
       20:34:30.622 [main] DEBUG org.apache.spark.sql.SparkSession - Ignored 
static SQL configurations:
         spark.sql.warehouse.dir=2
       20:34:30.623 [main] DEBUG org.apache.spark.sql.SparkSession - Other 
ignored configurations:
         spark.abcd=abcb4
         spark.abc=abcb
       ```
   
   2. Do not show a configuration already explicitly set
   
       ```bash
       ./bin/spark-shell --conf spark.abc=abc
       ```
   
       ```scala
       import org.apache.spark.sql.SparkSession
       spark.sparkContext.setLogLevel("DEBUG")
       SparkSession.builder.config("spark.abc", "abc").getOrCreate  // 
**Ignore** warnings because it's already set in --conf
       SparkSession.builder.config("spark.abc.new", "abc").getOrCreate  // 
**Show** warnings for only configuration newly set.
       SparkSession.builder.config("spark.abc.new", "abc").getOrCreate  // 
**Ignore** warnings because it's set ^.
       ```
   
       **Before**:
   
       ```
       21:13:56.183 [main] WARN  org.apache.spark.sql.SparkSession - Using an 
existing SparkSession; some spark core configurations may not take effect.
       21:13:56.356 [main] WARN  org.apache.spark.sql.SparkSession - Using an 
existing SparkSession; some spark core configurations may not take effect.
       21:13:56.476 [main] WARN  org.apache.spark.sql.SparkSession - Using an 
existing SparkSession; some spark core configurations may not take effect.
       ```
   
   
       **After:**
   
       ```
       20:36:36.251 [main] WARN  org.apache.spark.sql.SparkSession - Using an 
existing Spark session; only runtime SQL configurations will take effect.
       20:36:36.253 [main] DEBUG org.apache.spark.sql.SparkSession - Other 
ignored configurations:
         spark.abc.new=abc
       ```
   
   Note that there is no behaviour change on session state initialization when 
configurations are not set. For example:
   
   ```scala
   import org.apache.spark.sql.SparkSession
   spark.sparkContext.setLogLevel("DEBUG")
   SparkSession.builder.getOrCreate
   ```
   
   But the session state initialization can be triggered even for static SQL 
configurations set after this PR. Previously, it was not triggered. This would 
not introduce something user-facing or a bug but worth noting it.
   For runtime SQL configurations, the session state initialization in this 
code path was introduced at https://github.com/apache/spark/pull/15295.
   
   ### How was this patch tested?
   
   It was manually tested as shown above.


-- 
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.

To unsubscribe, e-mail: [email protected]

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