-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-23706][PYTHON] spark.conf.get(value, default=None) should produce None in PySpark #20841
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
Conversation
|
cc @cloud-fan, @viirya, @ueshin, @BryanCutler who I can directly think of for now. |
|
Test build #88291 has finished for PR 20841 at commit
|
|
retest this please |
|
Test build #88294 has finished for PR 20841 at commit
|
python/pyspark/sql/context.py
Outdated
| def getConf(self, key, defaultValue=_NoValue): | ||
| """Returns the value of Spark SQL configuration property for the given key. | ||
| If the key is not set and defaultValue is not None, return |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We should update this doc.
|
Test build #88298 has finished for PR 20841 at commit
|
|
retest this please. |
|
Test build #88300 has finished for PR 20841 at commit
|
ueshin
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM except for one comment.
|
|
||
| self.assertRaisesRegexp(Exception, "hyukjin", lambda: spark.conf.get("hyukjin")) | ||
| self.assertEqual(spark.conf.get("hyukjin", None), None) | ||
| self.assertEqual(spark.conf.get("spark.sql.sources.partitionOverwriteMode", None), None) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you add one more test to get "spark.sql.sources.partitionOverwriteMode" without None to see the difference between with and without None? Hopefully with a simple comment.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure.
|
Shouldn't it be the same as in the application conf here https://github.com/apache/spark/blob/master/python/pyspark/conf.py#L174? Here the default is |
|
I think we should better match Python side behaviour to Scala side in general and throw a better exception if possible. I think the application conf was done like that because there wasn't BTW, could I ask to fix it separately please? I actually have found several places that we might have to consider |
|
Yeah, true that is how the Scala side works so I suppose that is best, but I kind of view this api similar to the python |
|
Ah, I got the point. So, you mean a dictionary-like approach - Strictly, I blieve this PR doesn't change that behaviour if I understood correctly but just preserves it anyway. |
python/pyspark/sql/tests.py
Outdated
| spark.conf.unset("bogo") | ||
| self.assertEqual(spark.conf.get("bogo", "colombia"), "colombia") | ||
|
|
||
| self.assertRaisesRegexp(Exception, "hyukjin", lambda: spark.conf.get("hyukjin")) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@BryanCutler, so this test case prompted you. It's not related with this PR:
Before
>>> spark.conf.get("hyukjin")
...
: java.util.NoSuchElementException: hyukjin
...
After
>>> spark.conf.get("hyukjin")
...
: java.util.NoSuchElementException: hyukjin
...
Let me take this out.
|
Test build #88336 has finished for PR 20841 at commit
|
|
retest this please |
|
Test build #88339 has finished for PR 20841 at commit
|
BryanCutler
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
|
LGTM. |
|
Merged to master and branch-2.3. Thank you @ueshin, @BryanCutler and @viirya for reviewing this. |
…uce None in PySpark
Scala:
```
scala> spark.conf.get("hey", null)
res1: String = null
```
```
scala> spark.conf.get("spark.sql.sources.partitionOverwriteMode", null)
res2: String = null
```
Python:
**Before**
```
>>> spark.conf.get("hey", None)
...
py4j.protocol.Py4JJavaError: An error occurred while calling o30.get.
: java.util.NoSuchElementException: hey
...
```
```
>>> spark.conf.get("spark.sql.sources.partitionOverwriteMode", None)
u'STATIC'
```
**After**
```
>>> spark.conf.get("hey", None) is None
True
```
```
>>> spark.conf.get("spark.sql.sources.partitionOverwriteMode", None) is None
True
```
*Note that this PR preserves the case below:
```
>>> spark.conf.get("spark.sql.sources.partitionOverwriteMode")
u'STATIC'
```
Manually tested and unit tests were added.
Author: hyukjinkwon <[email protected]>
Closes #20841 from HyukjinKwon/spark-conf-get.
(cherry picked from commit 61487b3)
Signed-off-by: hyukjinkwon <[email protected]>
…uce None in PySpark
## What changes were proposed in this pull request?
Scala:
```
scala> spark.conf.get("hey", null)
res1: String = null
```
```
scala> spark.conf.get("spark.sql.sources.partitionOverwriteMode", null)
res2: String = null
```
Python:
**Before**
```
>>> spark.conf.get("hey", None)
...
py4j.protocol.Py4JJavaError: An error occurred while calling o30.get.
: java.util.NoSuchElementException: hey
...
```
```
>>> spark.conf.get("spark.sql.sources.partitionOverwriteMode", None)
u'STATIC'
```
**After**
```
>>> spark.conf.get("hey", None) is None
True
```
```
>>> spark.conf.get("spark.sql.sources.partitionOverwriteMode", None) is None
True
```
*Note that this PR preserves the case below:
```
>>> spark.conf.get("spark.sql.sources.partitionOverwriteMode")
u'STATIC'
```
## How was this patch tested?
Manually tested and unit tests were added.
Author: hyukjinkwon <[email protected]>
Closes apache#20841 from HyukjinKwon/spark-conf-get.
…uce None in PySpark
Scala:
```
scala> spark.conf.get("hey", null)
res1: String = null
```
```
scala> spark.conf.get("spark.sql.sources.partitionOverwriteMode", null)
res2: String = null
```
Python:
**Before**
```
>>> spark.conf.get("hey", None)
...
py4j.protocol.Py4JJavaError: An error occurred while calling o30.get.
: java.util.NoSuchElementException: hey
...
```
```
>>> spark.conf.get("spark.sql.sources.partitionOverwriteMode", None)
u'STATIC'
```
**After**
```
>>> spark.conf.get("hey", None) is None
True
```
```
>>> spark.conf.get("spark.sql.sources.partitionOverwriteMode", None) is None
True
```
*Note that this PR preserves the case below:
```
>>> spark.conf.get("spark.sql.sources.partitionOverwriteMode")
u'STATIC'
```
Manually tested and unit tests were added.
Author: hyukjinkwon <[email protected]>
Closes apache#20841 from HyukjinKwon/spark-conf-get.
(cherry picked from commit 61487b3)
Signed-off-by: hyukjinkwon <[email protected]>
What changes were proposed in this pull request?
Scala:
Python:
Before
After
*Note that this PR preserves the case below:
How was this patch tested?
Manually tested and unit tests were added.