Skip to content

Conversation

@cloud-fan
Copy link
Contributor

@cloud-fan cloud-fan commented Sep 29, 2016

What changes were proposed in this pull request?

SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the schemaStringThreshold in HiveExternalCatalog, the flag to enable/disable hive support, the global temp view database in #14897.

Actually we've already implemented static SQL conf implicitly via SparkConf, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or SparkSession.conf, and forbid users to set/unset static SQL conf.

How was this patch tested?

new tests in SQLConfSuite

@cloud-fan
Copy link
Contributor Author

cc @yhuai @gatorsmile @liancheng

@SparkQA
Copy link

SparkQA commented Sep 29, 2016

Test build #66094 has finished for PR 15295 at commit ea2a57f.

  • This patch fails to build.
  • 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.conf.set(k, v) }
options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) }
Copy link
Contributor

Choose a reason for hiding this comment

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

this is change related?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yea, the goal is to forbid users to set/unset global sql conf via public API, but here we do need to set global sql conf with sparkConf, so we should use internal API here instead.

@andrewor14
Copy link
Contributor

LGTM. Pretty straightforward.

@SparkQA
Copy link

SparkQA commented Sep 29, 2016

Test build #66102 has finished for PR 15295 at commit 59fc4e9.

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

@cloud-fan
Copy link
Contributor Author

retest this please

@SparkQA
Copy link

SparkQA commented Sep 29, 2016

Test build #66109 has finished for PR 15295 at commit 59fc4e9.

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

@andrewor14
Copy link
Contributor

retest this please

@SparkQA
Copy link

SparkQA commented Sep 29, 2016

Test build #66119 has finished for PR 15295 at commit 59fc4e9.

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


test("cannot set/unset global SQL conf") {
intercept[AnalysisException](sql(s"SET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}=10"))
intercept[AnalysisException](sql(s"UNSET ${SCHEMA_STRING_LENGTH_THRESHOLD.key}=10"))
Copy link
Member

@gatorsmile gatorsmile Sep 29, 2016

Choose a reason for hiding this comment

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

We do not support UNSET command in SQL interface.

Maybe you can change it to

spark.conf.unset(SCHEMA_STRING_LENGTH_THRESHOLD.key)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

oh damn, but why test passed... it should throw parser exception

Copy link
Member

Choose a reason for hiding this comment

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

ParseException extends AnalysisException

* @since 2.0.0
*/
def set(key: String, value: String): Unit = {
assertNotGlobalSQLConf(key)
Copy link
Member

@gatorsmile gatorsmile Sep 29, 2016

Choose a reason for hiding this comment

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

I am wondering if we need to add this assertNotGlobalSQLConf in all the public set functions? I am just afraid we might forget to update the other set functions when adding more global sqlconf.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

all global sql conf will be automatically registered, we don't need to change the code here when adding more global sql conf

Copy link
Member

Choose a reason for hiding this comment

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

Sorry, I did not explain it clearly. What I meant above is to add assertNotGlobalSQLConf to the other two set functions Does this make sense?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok, looks safer

Copy link
Contributor

Choose a reason for hiding this comment

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

should be assertNonStaticConf here

.checkValues(Set("hive", "in-memory"))
.createWithDefault("in-memory")

// This is used to control the when we will split a schema's JSON string to multiple pieces
Copy link
Member

@gatorsmile gatorsmile Sep 29, 2016

Choose a reason for hiding this comment

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

Nit: add max length

@rxin
Copy link
Contributor

rxin commented Sep 30, 2016

"Global" isn't the right name. The main property is immutability once the service starts, rather than the fact that they are shared by all the sessions.

}

test("SPARK-6024 wide schema support") {
withSQLConf(SQLConf.SCHEMA_STRING_LENGTH_THRESHOLD.key -> "4000") {
Copy link
Member

Choose a reason for hiding this comment

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

This line in the old test case is useless... : (

@rxin
Copy link
Contributor

rxin commented Sep 30, 2016

@srinathshankar suggested "static config"


private def assertNotGlobalSQLConf(key: String): Unit = {
if (GlobalSQLConf.globalConfKeys.contains(key)) {
throw new AnalysisException(s"Can not set/unset a global SQL conf: $key")
Copy link
Contributor

Choose a reason for hiding this comment

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

Cannot modify the value of a static config: $key

@SparkQA
Copy link

SparkQA commented Sep 30, 2016

Test build #66147 has finished for PR 15295 at commit e1a6898.

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

@SparkQA
Copy link

SparkQA commented Oct 1, 2016

Test build #66218 has finished for PR 15295 at commit 714f22e.

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

@SparkQA
Copy link

SparkQA commented Oct 3, 2016

Test build #66264 has finished for PR 15295 at commit d240d27.

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

@cloud-fan cloud-fan changed the title [SPARK-17720][SQL] introduce global SQL conf [SPARK-17720][SQL] introduce static SQL conf Oct 10, 2016
@SparkQA
Copy link

SparkQA commented Oct 10, 2016

Test build #66640 has finished for PR 15295 at commit fc78834.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 10, 2016

Test build #66660 has finished for PR 15295 at commit b4c1e0f.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Oct 11, 2016

Test build #66697 has finished for PR 15295 at commit 8d93c4a.

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

@cloud-fan
Copy link
Contributor Author

retest this please

@SparkQA
Copy link

SparkQA commented Oct 11, 2016

Test build #66706 has finished for PR 15295 at commit 8d93c4a.

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

@SparkQA
Copy link

SparkQA commented Oct 11, 2016

Test build #66722 has finished for PR 15295 at commit 0ad8815.

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

sqlConf.contains(key)
}

private def assertNotGlobalSQLConf(key: String): Unit = {
Copy link
Contributor

@rxin rxin Oct 11, 2016

Choose a reason for hiding this comment

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

rename to requireNonStaticConf?

@rxin
Copy link
Contributor

rxin commented Oct 11, 2016

LGTM except that one comment on naming.

@SparkQA
Copy link

SparkQA commented Oct 12, 2016

Test build #66777 has finished for PR 15295 at commit 595b220.

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

@gatorsmile
Copy link
Member

LGTM

@asfgit asfgit closed this in b9a1471 Oct 12, 2016
@gatorsmile
Copy link
Member

Merging to master! Thanks!

uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
## What changes were proposed in this pull request?

SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in apache#14897.

Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf.

## How was this patch tested?

new tests in SQLConfSuite

Author: Wenchen Fan <[email protected]>

Closes apache#15295 from cloud-fan/global-conf.
HyukjinKwon added a commit that referenced this pull request Dec 24, 2021
…y set in SparkSession.builder.getOrCreate

### 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 #34757 (comment).

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

Yes.

1. Show ignored configurations 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 - 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.

    ```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 - Configurations that might not take effect:
      spark.abc.new=abc
    ```

3. Do not issue a warning and hide runtime SQL configurations in debug log:

    ```bash
    ./bin/spark-shell
    ```

    ```scala
    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:

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

Closes #35001 from HyukjinKwon/SPARK-37727.

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

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants