Skip to content

Conversation

@HyukjinKwon
Copy link
Member

@HyukjinKwon HyukjinKwon commented Dec 23, 2021

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:

./bin/spark-shell --conf spark.abc=abc
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 #34757 (comment).

Does this PR introduce any user-facing change?

Yes.

  1. Show ignored configurations in debug level logs:

    ./bin/spark-shell --conf spark.abc=abc
    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 - Configurations that might not take effect:
      spark.abcd=abcb4
      spark.abc=abcb
    
  2. Do not issue a warning and hide a configuration already explicitly set (with the same value) before.

    ./bin/spark-shell --conf spark.abc=abc
    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 - Configurations that might not take effect:
      spark.abc.new=abc
    
  3. Do not issue a warning and hide runtime SQL configurations in debug log:

    ./bin/spark-shell
    import org.apache.spark.sql.SparkSession
    spark.sparkContext.setLogLevel("DEBUG")
    SparkSession.builder.config("spark.sql.ansi.enabled", "true").getOrCreate  // **Ignore** warnings for runtime SQL configurations
    SparkSession.builder.config("spark.buffer.size", "1234").getOrCreate  // **Show** warnings for Spark core configuration
    SparkSession.builder.config("spark.sql.source.specific", "abc").getOrCreate  // **Show** warnings for custom runtime options
    SparkSession.builder.config("spark.sql.warehouse.dir", "xyz").getOrCreate  // **Show** warnings for static SQL configurations

    Before:

    11:11:40.846 [main] WARN  org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect.
    11:11:41.037 [main] WARN  org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect.
    11:11:41.167 [main] WARN  org.apache.spark.sql.SparkSession - Using an existing SparkSession; some spark core configurations may not take effect.
    11:11:41.318 [main] WARN  org.apache.spark.sql.SparkSession - Using an existing SparkSession; the static sql configurations will not take effect.
    

    After:

    10:39:54.870 [main] WARN  org.apache.spark.sql.SparkSession - Using an existing Spark session; only runtime SQL configurations will take effect.
    10:39:54.872 [main] DEBUG org.apache.spark.sql.SparkSession - Configurations that might not take effect:
      spark.buffer.size=1234
    10:39:54.988 [main] WARN  org.apache.spark.sql.SparkSession - Using an existing Spark session; only runtime SQL configurations will take effect.
    10:39:54.988 [main] DEBUG org.apache.spark.sql.SparkSession - Configurations that might not take effect:
      spark.sql.source.specific=abc
    10:39:55.107 [main] WARN  org.apache.spark.sql.SparkSession - Using an existing Spark session; only runtime SQL configurations will take effect.
    10:39:55.108 [main] DEBUG org.apache.spark.sql.SparkSession - Ignored static SQL configurations:
      spark.sql.warehouse.dir=xyz
    

Note that there is no behaviour change on session state initialization when configurations are not set. For example:

import org.apache.spark.sql.SparkSession
spark.sparkContext.setLogLevel("DEBUG")
SparkSession.builder.getOrCreate

But the session state initialization can be triggered now 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 #15295.

How was this patch tested?

It was manually tested as shown above.

@github-actions github-actions bot added the SQL label Dec 23, 2021
@HyukjinKwon
Copy link
Member Author

cc @cloud-fan, @dongjoon-hyun @AngersZhuuuu @yaooqinn @maropu FYI when you find some time to review :-).

@SparkQA
Copy link

SparkQA commented Dec 23, 2021

@SparkQA
Copy link

SparkQA commented Dec 23, 2021

Copy link
Member

@yaooqinn yaooqinn left a comment

Choose a reason for hiding this comment

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

LGTM

@Ngone51
Copy link
Member

Ngone51 commented Dec 23, 2021

20:36:36.253 [main] DEBUG org.apache.spark.sql.SparkSession - Configurations that might not take effect:
spark.abc.new=abc

I feel this could be a bit confused. Users may thought the value abc doesn't take effect for the conf spark.abc.new. However, it does. The thing is, it's actually the second set doesn't take effect here rather than the value itself.

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

+1, LGTM. Thank you, @HyukjinKwon .

@SparkQA
Copy link

SparkQA commented Dec 23, 2021

Test build #146522 has finished for PR 35001 at commit f58f6c3.

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

@SparkQA
Copy link

SparkQA commented Dec 23, 2021

Test build #146527 has finished for PR 35001 at commit f710487.

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

@HyukjinKwon
Copy link
Member Author

I feel this could be a bit confused. Users may thought the value abc doesn't take effect for the conf spark.abc.new. However, it does. The thing is, it's actually the second set doesn't take effect here rather than the value itself.

This actually was already there. The problem here is that users, for example, might pass some source specific options that Spark side does not now. Maybe we can filter out Spark's runtime configuration here though. Let me take a look while I am here.

@HyukjinKwon
Copy link
Member Author

@Ngone51, the current change should address your comment (see 3. in Pr desscription)

@AngersZhuuuu
Copy link
Contributor

Maybe we can have a UT in SparkShellSuite?

@HyukjinKwon
Copy link
Member Author

Maybe we can have a UT in SparkShellSuite?

There is actually a bug I guess from switching to log4j 2 (see SPARK-37729). I would prefer to add it after this gets resolved.

@AngersZhuuuu
Copy link
Contributor

Maybe we can have a UT in SparkShellSuite?

There is actually a bug I guess from switching to log4j 2 (see SPARK-37729). I would prefer to add it after this gets resolved.

Got it, thanks.

Copy link
Contributor

@AngersZhuuuu AngersZhuuuu left a comment

Choose a reason for hiding this comment

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

LGTM

@SparkQA
Copy link

SparkQA commented Dec 24, 2021

@SparkQA
Copy link

SparkQA commented Dec 24, 2021

@HyukjinKwon
Copy link
Member Author

Merged to master.

@HyukjinKwon
Copy link
Member Author

Thanks guys for your reviews.

@Ngone51
Copy link
Member

Ngone51 commented Dec 24, 2021

late lgtm

@SparkQA
Copy link

SparkQA commented Dec 24, 2021

Test build #146543 has finished for PR 35001 at commit ae4616c.

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

HyukjinKwon added a commit that referenced this pull request Dec 29, 2021
…Session.builder.getOrCreate

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

This PR adds some test cases for #35001. Test cases were not added because there are related bugs but they are fixed now.

### Why are the changes needed?

To avoid regressions.

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

No, dev-only.

### How was this patch tested?

This PR adds test cases that were manually tested in #35001.

Closes #35048 from HyukjinKwon/SPARK-37727-followup.

Authored-by: Hyukjin Kwon <[email protected]>
Signed-off-by: Hyukjin Kwon <[email protected]>
@HyukjinKwon HyukjinKwon deleted the SPARK-37727 branch January 4, 2022 00:51
asiunov pushed a commit to ascend-io/spark that referenced this pull request Aug 25, 2022
…Session.builder.getOrCreate

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

This PR adds some test cases for apache#35001. Test cases were not added because there are related bugs but they are fixed now.

### Why are the changes needed?

To avoid regressions.

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

No, dev-only.

### How was this patch tested?

This PR adds test cases that were manually tested in apache#35001.

Closes apache#35048 from HyukjinKwon/SPARK-37727-followup.

Authored-by: Hyukjin Kwon <[email protected]>
Signed-off-by: Hyukjin Kwon <[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.

6 participants