-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-37504][PYTHON] Pyspark create SparkSession with existed session should not pass static conf #34757
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-37504][PYTHON] Pyspark create SparkSession with existed session should not pass static conf #34757
Changes from all commits
800a184
a91cdc9
85f9188
8a0f79c
5b406f5
bc644ff
00b4c9a
2889e1f
a7f4a51
af2185f
93c429e
f6f06cc
47480cf
8e66ec4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -277,8 +277,10 @@ def getOrCreate(self) -> "SparkSession": | |
| # Do not update `SparkConf` for existing `SparkContext`, as it's shared | ||
| # by all sessions. | ||
| session = SparkSession(sc, options=self._options) | ||
| for key, value in self._options.items(): | ||
| session._jsparkSession.sessionState().conf().setConfString(key, value) | ||
| else: | ||
| getattr( | ||
| getattr(session._jvm, "SparkSession$"), "MODULE$" | ||
| ).applyModifiableSettings(session._jsparkSession, self._options) | ||
| return session | ||
|
|
||
| builder = Builder() | ||
|
|
@@ -291,7 +293,7 @@ def __init__( | |
| self, | ||
| sparkContext: SparkContext, | ||
| jsparkSession: Optional[JavaObject] = None, | ||
| options: Optional[Dict[str, Any]] = {}, | ||
| options: Dict[str, Any] = {}, | ||
| ): | ||
| from pyspark.sql.context import SQLContext | ||
|
|
||
|
|
@@ -304,8 +306,15 @@ def __init__( | |
| and not self._jvm.SparkSession.getDefaultSession().get().sparkContext().isStopped() | ||
| ): | ||
| jsparkSession = self._jvm.SparkSession.getDefaultSession().get() | ||
| getattr(getattr(self._jvm, "SparkSession$"), "MODULE$").applyModifiableSettings( | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @AngersZhuuuu, this actually shows a lot of new warnings (see also #34893). Another reproducer: ./bin/spark-shell --conf spark.executor.memory=8g --conf spark.driver.memory=8g>>> from pyspark.sql.functions import udf
>>> udf(lambda x: x)("a")
21/12/15 14:03:15 WARN SparkSession: Using an existing SparkSession; the static sql configurations will not take effect.
Column<'<lambda>(a)'>There are more places to fix like this: ml/util.py: self._sparkSession = SparkSession.builder.getOrCreate()
sql/column.py: spark = SparkSession.builder.getOrCreate()
sql/context.py: sparkSession = SparkSession.builder.getOrCreate()
sql/readwriter.py: spark = SparkSession.builder.getOrCreate()
sql/readwriter.py: spark = SparkSession.builder.getOrCreate()
sql/session.py: return SparkSession.builder.getOrCreate()
sql/session.py: return SparkSession.builder.getOrCreate()
sql/streaming.py: spark = SparkSession.builder.getOrCreate()
sql/streaming.py: spark = SparkSession.builder.getOrCreate()
sql/udf.py: spark = SparkSession.builder.getOrCreate()
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we can't make it in Spark 3.3, I think maybe it's just safer to revert #34757 #34732 and #34559 for now because each patch here will introduce either:
Separately, I still feel 8424f55 is inefficient. We don't know which configurations don't take effect, or why it keeps complaining (see the example above) for which configuration. We should probably at least print out the keys or lower the level of log.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @xinrong-databricks actually this is more Python side codes. Are you interested in creating a followup?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
+1
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thank you for the head-ups, @HyukjinKwon .
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Certainly, I will fix it and keep you updated. Thanks!
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
+1
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Made a PR at #35001 👍 |
||
| jsparkSession, options | ||
| ) | ||
| else: | ||
| jsparkSession = self._jvm.SparkSession(self._jsc.sc(), options) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shall we add a short comment here that this is the case when we can set static configurations |
||
| else: | ||
| getattr(getattr(self._jvm, "SparkSession$"), "MODULE$").applyModifiableSettings( | ||
| jsparkSession, options | ||
| ) | ||
| self._jsparkSession = jsparkSession | ||
| self._jwrapped = self._jsparkSession.sqlContext() | ||
| self._wrapped = SQLContext(self._sc, self, self._jwrapped) | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.