[
https://issues.apache.org/jira/browse/ARROW-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16396524#comment-16396524
]
Wes McKinney commented on ARROW-2082:
-------------------------------------
I haven't found the bad memory access yet -- a pointer has gotten corrupted and
the segfault is happening there:
{code}
Thread 1 "python" received signal SIGSEGV, Segmentation fault.
0x00007ffff268f769 in arrow::PoolBuffer::Reserve (this=0xdfad80, capacity=1024)
at ../src/arrow/buffer.cc:101
101 RETURN_NOT_OK(pool_->Allocate(new_capacity, &new_data));
(gdb) p pool_
$1 = (arrow::MemoryPool *) 0x7ffff2d74300 <completed>
(gdb) p ::arrow::default_memory_pool()
$2 = (arrow::MemoryPool *) 0x7ffff2d74310
<arrow::default_memory_pool()::default_memory_pool_>
{code}
Those memory addresses should be the same. Here is a minimal script repro'ing
the error:
{code:language=python}
import pandas as pd
import pyarrow as pa
import pyarrow.parquet as pq
from io import StringIO
content = StringIO(
"""report_time_of_generation
2018-02-01 03:44:29
2018-02-01 03:44:29
2018-02-01 03:44:29
2018-02-01 03:44:29
2018-02-01 03:44:29
""")
dfs = pd.read_csv(content, parse_dates=['report_time_of_generation'])
table = pa.Table.from_pandas(dfs)
pq.write_table(table, 'output.parquet', flavor='spark', compression='snappy',
coerce_timestamps='ms')
{code}
> [Python] SegFault in pyarrow.parquet.write_table with specific options
> ----------------------------------------------------------------------
>
> Key: ARROW-2082
> URL: https://issues.apache.org/jira/browse/ARROW-2082
> Project: Apache Arrow
> Issue Type: Bug
> Components: Python
> Affects Versions: 0.8.0
> Environment: tested on MacOS High Sierra with python 3.6 and Ubuntu
> Xenial (Python 3.5)
> Reporter: Clément Bouscasse
> Priority: Major
> Fix For: 0.9.0
>
>
> I originally filed an issue in the pandas project but we've tracked it down
> to arrow itself, when called via pandas in specific circumstances:
> [https://github.com/pandas-dev/pandas/issues/19493]
> basically using
> {code:java}
> df.to_parquet('filename.parquet', flavor='spark'){code}
> gives a seg fault if `df` contains a datetime column.
> Under the covers, pandas translates this to the following call:
> {code:java}
> pq.write_table(table, 'output.parquet', flavor='spark', compression='snappy',
> coerce_timestamps='ms')
> {code}
> which gives me an instant crash.
> There is a repro on the github ticket.
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)