Fix misleading TypeError for scalar overflow in dask.array elemwise#12301
Fix misleading TypeError for scalar overflow in dask.array elemwise#12301jacobtomlinson merged 5 commits intodask:mainfrom
Conversation
When dtype inference fails in elemwise due to a scalar overflow (e.g. int8 * 128), Dask previously returned NotImplemented. This caused Python to fall back to the scalar reverse-operator and raise a misleading TypeError. Preserve the original exception via chaining in apply_infer_dtype and re-raise OverflowError in elemwise when that is the root cause. Add regression test for int8 scalar overflow.
Unit Test ResultsSee test report for an extended history of previous test failures. This is useful for diagnosing flaky tests. 21 files ± 0 21 suites ±0 5h 45m 9s ⏱️ + 15m 20s Results for commit 50f9cf3. ± Comparison against base commit ac02b7d. ♻️ This comment has been updated with latest results. |
|
The CI failure (
Fixed by checking both: |
jacobtomlinson
left a comment
There was a problem hiding this comment.
A couple of questions.
@lucascolley if you have some time you may also want to take a look.
Let OverflowError propagate directly from apply_infer_dtype instead of wrapping it in ValueError. Add a matching except clause in elemwise so the error is not swallowed by the broad Exception handler. Signed-off-by: Maxime Grenu <maxime.grenu@gmail.com>
083ccbe to
144bc8d
Compare
jacobtomlinson
left a comment
There was a problem hiding this comment.
Can you also look at the failing CI? Given the errors are in mindeps I expect older versions of numpy don't raise an OverflowError when scalars overflow, they probably do something else instead. We need to handle this case too.
NumPy < 2.0 silently upcasts out-of-bounds scalars instead of raising OverflowError, so the test cannot pass on those versions. Signed-off-by: Maxime Grenu <maxime.grenu@gmail.com>
|
thank you both! |
What this PR does
Fixes a confusing error when applying elemwise operations between a small integer-typed Dask array and a Python scalar that is out of bounds for the dtype.
Example:
Previously raised:
Now raises (matching NumPy semantics and message):
Why
dask.array.core.elemwiseperforms dtype inference viaapply_infer_dtype. When inference fails,elemwisereturnedNotImplemented, causing Python to fall back to the scalar reverse-op and emit a misleadingTypeError.This PR:
apply_infer_dtype(raise ValueError(...) from e)OverflowError) inelemwiseand re-raises itCloses #12296