-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
[C++] Take kernel can't handle ChunkedArrays that don't fit in an Array #25822
Comments
Antoine Pitrou / @pitrou: >>> import pyarrow as pa
>>> arr = pa.array(["x" * (1<<20)])
>>> t = arr.take(pa.array([0]*((1<<12) + 1), type=pa.int8()))
>>> t.validate(full=True)
Traceback (most recent call last):
[...]
ArrowInvalid: Offset invariant failure: non-monotonic offset at slot 2048: -2147483648 < 2146435072 |
Leonard Lausen / @leezu: |
Chris Fregly: import ray ray.init(address="auto") df = ray.data.read_parquet(" print(df.groupby("product_category").count()) Here's the error: |
Mayur Srivastava / @mayuropensource:
Thanks, Mayur Srivastava |
David Li / @lidavidm: Note that I think the cases described in the comments above are fundamentally different from the original issue: they also require upgrading the output from Array to ChunkedArray (or from String/List to LargeString/LargeList) and so can't be done automatically. |
Will Jones / @wjones127: |
David Li / @lidavidm: |
Will Jones / @wjones127: Problem 1: We concatenate when we shouldn't need toThis fails: arr = pa.chunked_array([["a" * 2**30]] * 2)
arr.take([0,1])
# Traceback (most recent call last):
# File "<stdin>", line 1, in <module>
# File "pyarrow/table.pxi", line 998, in pyarrow.lib.ChunkedArray.take
# File "/Users/willjones/Documents/test-env/venv/lib/python3.9/site-packages/pyarrow/compute.py", line 457, in take
# return call_function('take', [data, indices], options, memory_pool)
# File "pyarrow/_compute.pyx", line 542, in pyarrow._compute.call_function
# File "pyarrow/_compute.pyx", line 341, in pyarrow._compute.Function.call
# File "pyarrow/error.pxi", line 144, in pyarrow.lib.pyarrow_internal_check_status
# File "pyarrow/error.pxi", line 100, in pyarrow.lib.check_status
# pyarrow.lib.ArrowInvalid: offset overflow while concatenating arrays because we concatenate input values here. If that were corrected, it would then fail on the concatenation here if the indices were a chunked array. The first concatenation could be avoided somewhat easily in special cases (sorted / fall in same chunk), which was partially implement in R. For the general case, we'd need to address this within the kernel rather than within pre-processing (see Problem 3). The second concatenation shouldn't always be eliminated, but we might want to add a check to validate that there is enough room in offset buffers of arrays to concatenate. TBD if there is an efficient way to test that. Problem 2: take_array kernel doesn't handle case of offset overflowThis is what Antoine was pointing out: import pyarrow as pa
arr = pa.array(["x" * (1<<20)])
t = arr.take(pa.array([0]*((1<<12) + 1), type=pa.int8()))
t.validate(full=True)
# Traceback (most recent call last):
# [...]
# ArrowInvalid: Offset invariant failure: non-monotonic offset at slot 2048: -2147483648 < 2146435072 To solve this, I think we'd either have to:
|
Antoine Pitrou / @pitrou: I'd argue that this issue is primarily about fixing problem 3 (which would also fix problem 1). Besides correctness, concatenating is also a performance problem because we might be allocating a lot of temporary memory. |
David Li / @lidavidm: I would think we could avoid concatenation for all types, even if it isn't strictly required, to avoid excessive allocation as Antoine mentioned. |
Hi! Issue 3 is bothering us a lot when sorting huge tables. Do we have any updates on this? |
I'm working on this. |
@felipecrv @jorisvandenbossche do we have a clear line of sight when this issue will be addressed. Can you help me understand if there were/are any practical limitations around why chunking wasn't a consideration in the first place (for Table/ChunkedArray APIs)? This is a pretty foundational issue for For ex, in Ray Data
|
@alexeykudinkin take/filter on chunked arrays requires resolution of chunks which is more expensive than simple array offsetting [1]. Solutions that don't concatenate tend to be slower and are considered unacceptable. If it were up to me, I would leave the decision of concatenation to callers and never concatenate. I'm going to compromise and add conditional checks on sizes to decide if we concatenate or not. What I don't like about this solution is that it kinda doubles the amount of tests we need to run to cover both algorithms. [1] #41700 |
Take() currently concatenates ChunkedArrays first. However, this breaks down when calling Take() from a ChunkedArray or Table where concatenating the arrays would result in an array that's too large. While inconvenient to implement, it would be useful if this case were handled.
This could be done as a higher-level wrapper around Take(), perhaps.
Example in Python:
In this example, it would be useful if Take() or a higher-level wrapper could generate multiple record batches as output.
Reporter: Will Jones / @wjones127
Assignee: Will Jones / @wjones127
Related issues:
take
on a memory-mapped table #37766 (relates to)PRs and other links:
Note: This issue was originally created as ARROW-9773. Please see the migration documentation for further details.
The text was updated successfully, but these errors were encountered: