Skip to content

Conversation

@yaooqinn
Copy link
Member

@yaooqinn yaooqinn commented Mar 24, 2020

What changes were proposed in this pull request?

Currently, ResetCommand clear all configurations, including sql configs, static sql configs and spark context level configs.
for example:

spark-sql> set xyz=abc;
xyz abc
spark-sql> set;
spark.app.id local-1585055396930
spark.app.name SparkSQL::10.242.189.214
spark.driver.host 10.242.189.214
spark.driver.port 65094
spark.executor.id driver
spark.jars
spark.master local[*]
spark.sql.catalogImplementation hive
spark.sql.hive.version 1.2.1
spark.submit.deployMode client
xyz abc
spark-sql> reset;
spark-sql> set;
spark-sql> set spark.sql.hive.version;
spark.sql.hive.version 1.2.1
spark-sql> set spark.app.id;
spark.app.id <undefined>

In this PR, we restore spark confs to RuntimeConfig after it is cleared

Why are the changes needed?

reset command overkills configs which are static.

Does this PR introduce any user-facing change?

yes, the ResetCommand do not change static configs now

How was this patch tested?

add ut

@SparkQA
Copy link

SparkQA commented Mar 24, 2020

Test build #120275 has finished for PR 28003 at commit 9de836e.

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

@SparkQA
Copy link

SparkQA commented Mar 24, 2020

Test build #120278 has finished for PR 28003 at commit 3b6b3aa.

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

@yaooqinn
Copy link
Member Author

also, cc @cloud-fan @gatorsmile thanks

unsetConf(entry.key)
}

def clear(): Unit = {
Copy link
Member Author

@yaooqinn yaooqinn Mar 25, 2020

Choose a reason for hiding this comment

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

This API has been exposed to users since 2.1, and makes the SQLConf hard to be restored to its initial state after calling spark.sessionState.conf.clear

Copy link
Contributor

Choose a reason for hiding this comment

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

SQLConf is internnal.

Copy link
Member Author

Choose a reason for hiding this comment

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

scala> spark.sessionState.conf.clear

scala> spark.sessionState.conf
res1: org.apache.spark.sql.internal.SQLConf = org.apache.spark.sql.internal.SQLConf@54e24511

show we make it private[sql] ?

Copy link
Contributor

Choose a reason for hiding this comment

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

We don't. All classes under org.apache.spark.sql.internal are internal and users can still use. Spark marks things as internal by contract.

Copy link
Member Author

Choose a reason for hiding this comment

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

OK.


### Description
Reset all the properties specific to the current session to their default values. After RESET command, executing SET command will output empty.
Reset any properties specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values.
Copy link
Contributor

@cloud-fan cloud-fan Mar 25, 2020

Choose a reason for hiding this comment

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

"any properties" -> "any runtime configurations"

}

def clear(): Unit = {
val reserved = settings.asScala.filterKeys(staticConfKeys.contains).toMap
Copy link
Contributor

Choose a reason for hiding this comment

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

does staticConfKeys include the core configs?

Copy link
Member Author

@yaooqinn yaooqinn Mar 25, 2020

Choose a reason for hiding this comment

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

it doesn't. If a user calls it, it will make the SparkConfs in SQLConf all gone

/**
* This command is for resetting SQLConf to the default values. Command that runs
* This command is for resetting SQLConf to the default values. Any configurations that were set
* via [[SetCommand]] will get reset to default value Command that runs
Copy link
Contributor

Choose a reason for hiding this comment

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

a dot before Command that runs?

@SparkQA
Copy link

SparkQA commented Mar 25, 2020

Test build #120309 has finished for PR 28003 at commit 435e411.

  • 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 Mar 25, 2020

Test build #120318 has finished for PR 28003 at commit bcc6786.

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

@yaooqinn
Copy link
Member Author

retest this please

def clear(): Unit = {
val reserved = settings.asScala.filterKeys(staticConfKeys.contains).toMap
settings.clear()
settings.putAll(reserved.asJava)
Copy link
Contributor

Choose a reason for hiding this comment

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

is this change necessary? static SQL configs can only be set with SparkConf. We can clear everything here, and the RESET command will add back the configs in SparkConf

Copy link
Member Author

Choose a reason for hiding this comment

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

yes. it is not necessary.

spark.sessionState.conf.clear()
// After clear, only overrideConfs used by unit test should be in the SQLConf.
assert(spark.conf.getAll === TestSQLContext.overrideConfs)
assert((spark.conf.getAll -- TestSQLContext.overrideConfs.keys).size < spark.conf.getAll.size)
Copy link
Contributor

Choose a reason for hiding this comment

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

is this still needed?

sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM upperCaseData"), Nil)
}

test("SET commands semantics using sql()") {
Copy link
Contributor

Choose a reason for hiding this comment

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

do we still need to change?

@SparkQA
Copy link

SparkQA commented Mar 25, 2020

Test build #120342 has finished for PR 28003 at commit 8363a14.

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

### Description
Reset all the properties specific to the current session to their default values. After RESET command, executing SET command will output empty.
Reset any runtime configurations specific to the current session which were set via the [SET](sql-ref-syntax-aux-conf-mgmt-set.html) command to their default values.

Copy link
Member

Choose a reason for hiding this comment

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

We need to update the comment below, too?

-- Reset "all the properties" specific to the current session to their default values.
RESET;

Copy link
Member Author

Choose a reason for hiding this comment

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

good catch!

}
}

test("reset - static and spark conf") {
Copy link
Member

Choose a reason for hiding this comment

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

nit: the test name had better be self-descriptive.

@SparkQA
Copy link

SparkQA commented Mar 25, 2020

Test build #120346 has finished for PR 28003 at commit 45c926e.

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

@yaooqinn
Copy link
Member Author

retest this please

val warehouseDir = conf.get("spark.sql.warehouse.dir")
// ensure the conf here is not default value, and will not be reset to default value later
assert(warehouseDir.get.contains(this.getClass.getCanonicalName))
sql(s"reset")
Copy link
Contributor

Choose a reason for hiding this comment

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

no need the s

Copy link
Contributor

Choose a reason for hiding this comment

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

and it's better to upper case the SQL keyword.

@SparkQA
Copy link

SparkQA commented Mar 25, 2020

Test build #120338 has finished for PR 28003 at commit 69a4b82.

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

@SparkQA
Copy link

SparkQA commented Mar 25, 2020

Test build #120325 has finished for PR 28003 at commit bcc6786.

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

@SparkQA
Copy link

SparkQA commented Mar 25, 2020

Test build #120349 has finished for PR 28003 at commit 45c926e.

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

@SparkQA
Copy link

SparkQA commented Mar 25, 2020

Test build #120357 has finished for PR 28003 at commit d374c57.

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

@maropu
Copy link
Member

maropu commented Mar 25, 2020

Does this PR introduce any user-facing change?
yes, the ResetCommand do not change static configs now

btw, any chance to silently change behaviour of existing apps by this fix? We don't need to keep the current behaviour by a legacy config? cc: @gatorsmile @dongjoon-hyun @HyukjinKwon

@cloud-fan
Copy link
Contributor

I think this is clearly a bug. If users set a static SQL config in the config file, this should not be affected by SET or RESET.

@cloud-fan
Copy link
Contributor

thanks, merging to master/3.0!

@cloud-fan cloud-fan closed this in 44bd36a Mar 26, 2020
cloud-fan pushed a commit that referenced this pull request Mar 26, 2020
### What changes were proposed in this pull request?
Currently, ResetCommand clear all configurations, including sql configs, static sql configs and spark context level configs.
for example:
```sql
spark-sql> set xyz=abc;
xyz abc
spark-sql> set;
spark.app.id local-1585055396930
spark.app.name SparkSQL::10.242.189.214
spark.driver.host 10.242.189.214
spark.driver.port 65094
spark.executor.id driver
spark.jars
spark.master local[*]
spark.sql.catalogImplementation hive
spark.sql.hive.version 1.2.1
spark.submit.deployMode client
xyz abc
spark-sql> reset;
spark-sql> set;
spark-sql> set spark.sql.hive.version;
spark.sql.hive.version 1.2.1
spark-sql> set spark.app.id;
spark.app.id <undefined>
```
In this PR, we restore spark confs to  RuntimeConfig after it is cleared

### Why are the changes needed?
reset command overkills configs which are static.
### Does this PR introduce any user-facing change?

yes, the ResetCommand do not change static configs now

### How was this patch tested?

add ut

Closes #28003 from yaooqinn/SPARK-31234.

Authored-by: Kent Yao <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
(cherry picked from commit 44bd36a)
Signed-off-by: Wenchen Fan <[email protected]>
@gatorsmile
Copy link
Member

@yaooqinn Could you backport this to 2.4? and add it to the migration guide since it could impact the query results?

@yaooqinn
Copy link
Member Author

yaooqinn commented Apr 1, 2020

@yaooqinn Could you backport this to 2.4? and add it to the migration guide since it could impact the query results?

Hi @gatorsmile, why the migration guide should I put? In the backport or a followup to master?

sjincho pushed a commit to sjincho/spark that referenced this pull request Apr 15, 2020
### What changes were proposed in this pull request?
Currently, ResetCommand clear all configurations, including sql configs, static sql configs and spark context level configs.
for example:
```sql
spark-sql> set xyz=abc;
xyz abc
spark-sql> set;
spark.app.id local-1585055396930
spark.app.name SparkSQL::10.242.189.214
spark.driver.host 10.242.189.214
spark.driver.port 65094
spark.executor.id driver
spark.jars
spark.master local[*]
spark.sql.catalogImplementation hive
spark.sql.hive.version 1.2.1
spark.submit.deployMode client
xyz abc
spark-sql> reset;
spark-sql> set;
spark-sql> set spark.sql.hive.version;
spark.sql.hive.version 1.2.1
spark-sql> set spark.app.id;
spark.app.id <undefined>
```
In this PR, we restore spark confs to  RuntimeConfig after it is cleared

### Why are the changes needed?
reset command overkills configs which are static.
### Does this PR introduce any user-facing change?

yes, the ResetCommand do not change static configs now

### How was this patch tested?

add ut

Closes apache#28003 from yaooqinn/SPARK-31234.

Authored-by: Kent Yao <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
dongjoon-hyun pushed a commit that referenced this pull request Apr 19, 2020
…figuration

### What changes were proposed in this pull request?
This PR is to backport the fix of #28003, add a migration guide, update the PR description and add an end-to-end test case.

Before this PR, the SQL `RESET` command will reset the values of static SQL configuration to the default and remove the cached values of Spark Context Configurations in the current session. This PR fixes the bugs. After this PR, the `RESET` command follows its definition and only updates the runtime SQL configuration values to the default.

### Why are the changes needed?
When we introduced the feature of Static SQL Configuration, we did not update the implementation of  SQL `RESET` command.

The static SQL configuration should not be changed by any command at runtime. However, the `RESET` command resets the values to the default. We should fix them.

### Does this PR introduce any user-facing change?
Before Spark 2.4.6, the `RESET` command resets both the runtime and static SQL configuration values to the default. It also removes the cached values of Spark Context Configurations in the current session, although these configuration values are for displaying/querying only.

### How was this patch tested?
Added an end-to-end test and a unit test

Closes #28262 from gatorsmile/spark-31234followup2.4.

Authored-by: gatorsmile <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
gatorsmile added a commit that referenced this pull request Apr 20, 2020
…QL Configuration

### What changes were proposed in this pull request?
This PR is the follow-up PR of #28003

- add a migration guide
- add an end-to-end test case.

### Why are the changes needed?
The original PR made the major behavior change in the user-facing RESET command.

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

### How was this patch tested?
Added a new end-to-end test

Closes #28265 from gatorsmile/spark-31234followup.

Authored-by: gatorsmile <[email protected]>
Signed-off-by: gatorsmile <[email protected]>
gatorsmile added a commit that referenced this pull request Apr 20, 2020
…QL Configuration

### What changes were proposed in this pull request?
This PR is the follow-up PR of #28003

- add a migration guide
- add an end-to-end test case.

### Why are the changes needed?
The original PR made the major behavior change in the user-facing RESET command.

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

### How was this patch tested?
Added a new end-to-end test

Closes #28265 from gatorsmile/spark-31234followup.

Authored-by: gatorsmile <[email protected]>
Signed-off-by: gatorsmile <[email protected]>
(cherry picked from commit 6c792a7)
Signed-off-by: gatorsmile <[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