Skip to content

[SPARK-52570][PS] Enable divide-by-zero for numeric rmod with ANSI enabled #51275

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

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
28 changes: 4 additions & 24 deletions python/pyspark/pandas/data_type_ops/boolean_ops.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
import pandas as pd
from pandas.api.types import CategoricalDtype

from pyspark.pandas.base import column_op, IndexOpsMixin, numpy_column_op
from pyspark.pandas.base import column_op, IndexOpsMixin
from pyspark.pandas._typing import Dtype, IndexOpsLike, SeriesOrIndex
from pyspark.pandas.data_type_ops.base import (
DataTypeOps,
Expand All @@ -35,7 +35,6 @@
_is_boolean_type,
)
from pyspark.pandas.typedef.typehints import as_spark_type, extension_dtypes, pandas_on_spark_type
from pyspark.pandas.utils import is_ansi_mode_enabled
from pyspark.sql import functions as F, Column as PySparkColumn
from pyspark.sql.types import BooleanType, StringType
from pyspark.errors import PySparkValueError
Expand Down Expand Up @@ -137,21 +136,13 @@ def mod(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
raise TypeError(
"Modulo can not be applied to %s and the given type." % self.pretty_name
)
spark_session = left._internal.spark_frame.sparkSession

def safe_mod(left_col: PySparkColumn, right_val: Any) -> PySparkColumn:
if is_ansi_mode_enabled(spark_session):
Copy link
Contributor

Choose a reason for hiding this comment

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

Just curious why don't we need to check ansi_mode here anymore?

Copy link
Member Author

Choose a reason for hiding this comment

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

Bools are considered numeric and will take num_ops logic if that makes sense

Copy link
Member Author

Choose a reason for hiding this comment

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

Merged to unblock prs, please let me know if you have further concerns :)

return F.when(F.lit(right_val == 0), F.lit(None)).otherwise(left_col % right_val)
else:
return left_col % right_val

if isinstance(right, numbers.Number):
left = transform_boolean_operand_to_numeric(left, spark_type=as_spark_type(type(right)))
return numpy_column_op(safe_mod)(left, right)
return left % right
else:
assert isinstance(right, IndexOpsMixin)
left = transform_boolean_operand_to_numeric(left, spark_type=right.spark.data_type)
return numpy_column_op(safe_mod)(left, right)
return left % right

def pow(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
Expand Down Expand Up @@ -235,18 +226,7 @@ def rmod(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if isinstance(right, numbers.Number) and not isinstance(right, bool):
left = transform_boolean_operand_to_numeric(left, spark_type=as_spark_type(type(right)))
spark_session = left._internal.spark_frame.sparkSession

if is_ansi_mode_enabled(spark_session):

def safe_rmod(left_col: PySparkColumn, right_val: Any) -> PySparkColumn:
return F.when(left_col != 0, F.pmod(F.lit(right_val), left_col)).otherwise(
F.lit(None)
)

return numpy_column_op(safe_rmod)(left, right)
else:
return right % left
return right % left
else:
raise TypeError(
"Modulo can not be applied to %s and the given type." % self.pretty_name
Expand Down
17 changes: 13 additions & 4 deletions python/pyspark/pandas/data_type_ops/num_ops.py
Original file line number Diff line number Diff line change
Expand Up @@ -160,12 +160,21 @@ def rmod(self, left: IndexOpsLike, right: Any) -> SeriesOrIndex:
_sanitize_list_like(right)
if not isinstance(right, numbers.Number):
raise TypeError("Modulo can not be applied to given types.")

def rmod(left: PySparkColumn, right: Any) -> PySparkColumn:
return ((right % left) + left) % left
spark_session = left._internal.spark_frame.sparkSession

right = transform_boolean_operand_to_numeric(right)
return column_op(rmod)(left, right)

def safe_rmod(left_col: PySparkColumn, right_val: Any) -> PySparkColumn:
if is_ansi_mode_enabled(spark_session):
# Java-style modulo -> Python-style modulo
result = F.when(
left_col != 0, ((F.lit(right_val) % left_col) + left_col) % left_col
).otherwise(F.lit(None))
return result
else:
return ((right % left) + left) % left

return column_op(safe_rmod)(left, right)

def neg(self, operand: IndexOpsLike) -> IndexOpsLike:
return operand._with_new_scol(-operand.spark.column, field=operand._internal.data_fields[0])
Expand Down
3 changes: 2 additions & 1 deletion python/pyspark/pandas/tests/computation/test_binary_ops.py
Original file line number Diff line number Diff line change
Expand Up @@ -225,11 +225,12 @@ def test_binary_operator_floordiv(self):

def test_binary_operator_mod(self):
# Positive
pdf = pd.DataFrame({"a": [3], "b": [2]})
pdf = pd.DataFrame({"a": [3], "b": [2], "c": [0]})
Copy link
Member

Choose a reason for hiding this comment

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

Seems like this change broke the non ANSI mode:


======================================================================
ERROR [4.586s]: test_binary_operator_mod (pyspark.pandas.tests.computation.test_binary_ops.FrameBinaryOpsTests.test_binary_operator_mod)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/__w/spark/spark/python/pyspark/pandas/tests/computation/test_binary_ops.py", line 233, in test_binary_operator_mod
    self.assert_eq(1 % psdf["c"], 1 % pdf["c"])
                   ~~^~~~~~~~~~~
  File "/__w/spark/spark/python/pyspark/pandas/base.py", line 386, in __rmod__
    return self._dtype_op.rmod(self, other)
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/__w/spark/spark/python/pyspark/pandas/data_type_ops/num_ops.py", line 177, in rmod
    return column_op(safe_rmod)(left, right)
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/__w/spark/spark/python/pyspark/pandas/base.py", line 222, in wrapper
    scol = f(
           ^^
  File "/__w/spark/spark/python/pyspark/pandas/data_type_ops/num_ops.py", line 175, in safe_rmod
    return ((right % left) + left) % left
             ~~~~~~^~~~~~
  File "/__w/spark/spark/python/pyspark/pandas/base.py", line 386, in __rmod__
    return self._dtype_op.rmod(self, other)
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/__w/spark/spark/python/pyspark/pandas/data_type_ops/num_ops.py", line 177, in rmod
    return column_op(safe_rmod)(left, right)
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/__w/spark/spark/python/pyspark/pandas/base.py", line 222, in wrapper
    scol = f(
           ^^
  File "/__w/spark/spark/python/pyspark/pandas/data_type_ops/num_ops.py", line 175, in safe_rmod
    return ((right % left) + left) % left
             ~~~~~~^~~~~~
  File "/__w/spark/spark/python/pyspark/pandas/base.py", line 386, in __rmod__
    return self._dtype_op.rmod(self, other)
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/__w/spark/spark/python/pyspark/pandas/data_type_ops/num_ops.py", line 177, in rmod
    return column_op(safe_rmod)(left, right)
           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/__w/spark/spark/python/pyspark/pandas/base.py", line 222, in wrapper
    scol = f(
           ^^

https://github.com/apache/spark/actions/runs/15987607479/job/45094971366

Let me revert it for now.

psdf = ps.from_pandas(pdf)

self.assert_eq(psdf["a"] % psdf["b"], pdf["a"] % pdf["b"])
self.assert_eq(psdf["a"] % 0, pdf["a"] % 0)
self.assert_eq(1 % psdf["c"], 1 % pdf["c"])

# Negative
psdf = ps.DataFrame({"a": ["x"], "b": [1]})
Expand Down
13 changes: 5 additions & 8 deletions python/pyspark/pandas/tests/data_type_ops/test_boolean_ops.py
Original file line number Diff line number Diff line change
Expand Up @@ -235,16 +235,13 @@ def test_rpow(self):

def test_rmod(self):
psdf = self.psdf
pdf = self.pdf

b_psser = psdf["bool"]
# 1 % False is 0.0 in pandas
self.assert_eq(pd.Series([0, 0, None], dtype=float, name="bool"), 1 % b_psser)
# 0.1 / True is 0.1 in pandas
self.assert_eq(
pd.Series([0.10000000000000009, 0.10000000000000009, None], dtype=float, name="bool"),
0.1 % b_psser,
check_exact=False, # [0.1, 0.1, nan] for pandas-on-Spark
)
b_pser = pdf["bool"]
self.assert_eq(1 % b_pser.astype(float), 1 % b_psser)
# # Allow float precision diff: pandas: 0.10000000000000009; pandas on spark: 0.1
self.assert_eq(0.1 % b_pser, 0.1 % b_psser, almost=True)
self.assertRaises(TypeError, lambda: datetime.date(1994, 1, 1) % b_psser)
self.assertRaises(TypeError, lambda: True % b_psser)

Expand Down