You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "Antoine Pitrou (JIRA)" <ji...@apache.org> on 2018/03/12 12:39:00 UTC

[jira] [Commented] (ARROW-2227) [Python] Table.from_pandas does not create chunked_arrays.

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

Antoine Pitrou commented on ARROW-2227:
---------------------------------------

The snippet produces a core dump here (**). I think it is related to code touched by ARROW-2141 and especially the comment I posted here about an incorrect cast to 32-bit: https://github.com/apache/arrow/pull/1689/files#r173777819 .

(as a sidenote, I don't know why ARROW-2141 is necessary to allow conversion from Numpy but conversion from Pandas is already implemented. I suspect different paths are taken?)

(re-sidenote, where does the 2GB limit stem from? the desire to have shorter offset arrays?)

I would recommend deferring this to 0.10.0 so that we can sanitize the whole situation. There seem to be separate code paths converting Python bytes objects to Arrow data, with slightly different strategies...

(**) gdb backtrace:
{code}
#0  __memcpy_avx_unaligned () at ../sysdeps/x86_64/multiarch/memcpy-avx-unaligned.S:245
#1  0x00007fffc23dc5e8 in arrow::BufferBuilder::UnsafeAppend (this=0x7fffffffb538, data=0x7fff3f01d030, length=-2147483648)
    at /home/antoine/arrow/cpp/src/arrow/buffer.h:285
#2  0x00007fffc23dc236 in arrow::BufferBuilder::Append (this=0x7fffffffb538, data=0x7fff3f01d030, length=-2147483648)
    at /home/antoine/arrow/cpp/src/arrow/buffer.h:255
#3  0x00007fffc242b30f in arrow::TypedBufferBuilder<unsigned char>::Append (this=0x7fffffffb538, 
    arithmetic_values=0x7fff3f01d030 'x' <se r\377\377\377\377\377\377\377\377p\377\377\377\377\377\377\377\377te 200 fois>..., 
    num_elements=-2147483648) at /home/antoine/arrow/cpp/src/arrow/buffer.h:332
#4  0x00007fffc23d7f3e in arrow::BinaryBuilder::Append (this=0x7fffffffb4a0, 
    value=0x7fff3f01d030 'x' <se r\377\377\377\377\377\377\377\377p\377\377\377\377\377\377\377\377te 200 fois>..., length=-2147483648)
    at /home/antoine/arrow/cpp/src/arrow/builder.cc:1343
#5  0x00007fffc258d4b8 in arrow::BinaryBuilder::Append (this=0x7fffffffb4a0, 
    value=0x7fff3f01d030 'x' <se r\377\377\377\377\377\377\377\377p\377\377\377\377\377\377\377\377te 200 fois>..., length=-2147483648)
    at /home/antoine/arrow/cpp/src/arrow/builder.h:675
#6  0x00007fffc1f84923 in arrow::py::AppendObjectStrings (arr=0x7ffff7ecbee0, mask=0x0, offset=0, builder=0x7fffffffb4a0, 
    end_offset=0x7fffffffb480, have_bytes=0x7fffffffb430) at /home/antoine/arrow/cpp/src/arrow/python/numpy_to_arrow.cc:233
#7  0x00007fffc1f88d62 in arrow::py::NumPyConverter::ConvertObjectStrings (this=0x7fffffffbdd0)
    at /home/antoine/arrow/cpp/src/arrow/python/numpy_to_arrow.cc:859
#8  0x00007fffc1f8c22b in arrow::py::NumPyConverter::ConvertObjectsInfer (this=0x7fffffffbdd0)
    at /home/antoine/arrow/cpp/src/arrow/python/numpy_to_arrow.cc:1034
#9  0x00007fffc1f8d99c in arrow::py::NumPyConverter::ConvertObjects (this=0x7fffffffbdd0)
    at /home/antoine/arrow/cpp/src/arrow/python/numpy_to_arrow.cc:1135
#10 0x00007fffc1f85a2a in arrow::py::NumPyConverter::Convert (this=0x7fffffffbdd0)
    at /home/antoine/arrow/cpp/src/arrow/python/numpy_to_arrow.cc:504
#11 0x00007fffc1f9038d in arrow::py::NdarrayToArrow (pool=0x7fffc2a2d680 <arrow::default_memory_pool()::default_memory_pool_>, 
    ao=0x7ffff7ecbee0, mo=0x7ffff7d5ce90 <_Py_NoneStruct>, use_pandas_null_sentinels=true, type=..., out=0x7fffffffc0c0)
    at /home/antoine/arrow/cpp/src/arrow/python/numpy_to_arrow.cc:1577
{code}

> [Python] Table.from_pandas does not create chunked_arrays.
> ----------------------------------------------------------
>
>                 Key: ARROW-2227
>                 URL: https://issues.apache.org/jira/browse/ARROW-2227
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: Python
>    Affects Versions: 0.8.0
>            Reporter: Chris Ellison
>            Assignee: Wes McKinney
>            Priority: Major
>             Fix For: 0.9.0
>
>
> When creating a large enough array, pyarrow raises an exception:
> {code:java}
> import numpy as np
> import pandas as pd
> import pyarrow as pa
> x = list('1' * 2**31)
> y = pd.DataFrame({'x': x})
> t = pa.Table.from_pandas(y)
> # ArrowInvalid: BinaryArrow cannot contain more than 2147483646 bytes, have 2147483647{code}
> The array should be chunked for the user. As is, data frames with >2 GiB in binary data will struggle to get into arrow.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)