Skip to content

Commit f0169a1

Browse files
zero323gatorsmile
authored andcommitted
[SPARK-20290][MINOR][PYTHON][SQL] Add PySpark wrapper for eqNullSafe
## What changes were proposed in this pull request? Adds Python bindings for `Column.eqNullSafe` ## How was this patch tested? Manual tests, existing unit tests, doc build. Author: zero323 <[email protected]> Closes #17605 from zero323/SPARK-20290.
1 parent a355b66 commit f0169a1

File tree

2 files changed

+56
-1
lines changed

2 files changed

+56
-1
lines changed

python/pyspark/sql/column.py

Lines changed: 55 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -171,6 +171,61 @@ def __init__(self, jc):
171171
__ge__ = _bin_op("geq")
172172
__gt__ = _bin_op("gt")
173173

174+
_eqNullSafe_doc = """
175+
Equality test that is safe for null values.
176+
177+
:param other: a value or :class:`Column`
178+
179+
>>> from pyspark.sql import Row
180+
>>> df1 = spark.createDataFrame([
181+
... Row(id=1, value='foo'),
182+
... Row(id=2, value=None)
183+
... ])
184+
>>> df1.select(
185+
... df1['value'] == 'foo',
186+
... df1['value'].eqNullSafe('foo'),
187+
... df1['value'].eqNullSafe(None)
188+
... ).show()
189+
+-------------+---------------+----------------+
190+
|(value = foo)|(value <=> foo)|(value <=> NULL)|
191+
+-------------+---------------+----------------+
192+
| true| true| false|
193+
| null| false| true|
194+
+-------------+---------------+----------------+
195+
>>> df2 = spark.createDataFrame([
196+
... Row(value = 'bar'),
197+
... Row(value = None)
198+
... ])
199+
>>> df1.join(df2, df1["value"] == df2["value"]).count()
200+
0
201+
>>> df1.join(df2, df1["value"].eqNullSafe(df2["value"])).count()
202+
1
203+
>>> df2 = spark.createDataFrame([
204+
... Row(id=1, value=float('NaN')),
205+
... Row(id=2, value=42.0),
206+
... Row(id=3, value=None)
207+
... ])
208+
>>> df2.select(
209+
... df2['value'].eqNullSafe(None),
210+
... df2['value'].eqNullSafe(float('NaN')),
211+
... df2['value'].eqNullSafe(42.0)
212+
... ).show()
213+
+----------------+---------------+----------------+
214+
|(value <=> NULL)|(value <=> NaN)|(value <=> 42.0)|
215+
+----------------+---------------+----------------+
216+
| false| true| false|
217+
| false| false| true|
218+
| true| false| false|
219+
+----------------+---------------+----------------+
220+
221+
.. note:: Unlike Pandas, PySpark doesn't consider NaN values to be NULL.
222+
See the `NaN Semantics`_ for details.
223+
.. _NaN Semantics:
224+
https://spark.apache.org/docs/latest/sql-programming-guide.html#nan-semantics
225+
.. versionadded:: 2.3.0
226+
"""
227+
eqNullSafe = _bin_op("eqNullSafe", _eqNullSafe_doc)
228+
174229
# `and`, `or`, `not` cannot be overloaded in Python,
175230
# so use bitwise operators as boolean operators
176231
__and__ = _bin_op('and')

python/pyspark/sql/tests.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -982,7 +982,7 @@ def test_column_operators(self):
982982
cbool = (ci & ci), (ci | ci), (~ci)
983983
self.assertTrue(all(isinstance(c, Column) for c in cbool))
984984
css = cs.contains('a'), cs.like('a'), cs.rlike('a'), cs.asc(), cs.desc(),\
985-
cs.startswith('a'), cs.endswith('a')
985+
cs.startswith('a'), cs.endswith('a'), ci.eqNullSafe(cs)
986986
self.assertTrue(all(isinstance(c, Column) for c in css))
987987
self.assertTrue(isinstance(ci.cast(LongType()), Column))
988988
self.assertRaisesRegexp(ValueError,

0 commit comments

Comments
 (0)