You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@arrow.apache.org by "Antoine Pitrou (Jira)" <ji...@apache.org> on 2020/12/09 16:26:00 UTC
[jira] [Commented] (ARROW-9773) [C++] Take kernel can't handle
ChunkedArrays that don't fit in an Array
[ https://issues.apache.org/jira/browse/ARROW-9773?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17246649#comment-17246649 ]
Antoine Pitrou commented on ARROW-9773:
---------------------------------------
Note that this can happen with regular arrays too:
{code:python}
>>> 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
{code}
> [C++] Take kernel can't handle ChunkedArrays that don't fit in an Array
> -----------------------------------------------------------------------
>
> Key: ARROW-9773
> URL: https://issues.apache.org/jira/browse/ARROW-9773
> Project: Apache Arrow
> Issue Type: Improvement
> Components: C++
> Affects Versions: 1.0.0
> Reporter: David Li
> Priority: Major
>
> 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:
> {code:python}
> >>> import pyarrow as pa
> >>> pa.__version__
> '1.0.0'
> >>> rb1 = pa.RecordBatch.from_arrays([["a" * 2**30]], names=["a"])
> >>> rb2 = pa.RecordBatch.from_arrays([["b" * 2**30]], names=["a"])
> >>> table = pa.Table.from_batches([rb1, rb2], schema=rb1.schema)
> >>> table.take([1, 0])
> Traceback (most recent call last):
> File "<stdin>", line 1, in <module>
> File "pyarrow/table.pxi", line 1145, in pyarrow.lib.Table.take
> File "/home/lidavidm/Code/twosigma/arrow/venv/lib/python3.8/site-packages/pyarrow/compute.py", line 268, in take
> return call_function('take', [data, indices], options)
> File "pyarrow/_compute.pyx", line 298, in pyarrow._compute.call_function
> File "pyarrow/_compute.pyx", line 192, in pyarrow._compute.Function.call
> File "pyarrow/error.pxi", line 122, in pyarrow.lib.pyarrow_internal_check_status
> File "pyarrow/error.pxi", line 84, in pyarrow.lib.check_status
> pyarrow.lib.ArrowInvalid: offset overflow while concatenating arrays
> {code}
> In this example, it would be useful if Take() or a higher-level wrapper could generate multiple record batches as output.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)