[ 
https://issues.apache.org/jira/browse/ARROW-9773?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17551563#comment-17551563
 ] 

David Li commented on ARROW-9773:
---------------------------------

Agreed, we should focus on 1/3. Problem 2 is also interesting, but I'm not sure 
how best to handle it: right now the kernels infrastructure assumes a fixed 
output type and shape up front, and dynamically switching to ChunkedArray or 
promoting type would be a surprise.

I would think we could avoid concatenation for all types, even if it isn't 
strictly required, to avoid excessive allocation as Antoine mentioned.

> [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
>            Assignee: Will Jones
>            Priority: Major
>              Labels: kernel
>
> 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.20.7#820007)

Reply via email to