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

Wes McKinney commented on ARROW-3238:
-------------------------------------

It looks like there's an issue reading the last data page in fastparquet. See 
this gist. I created a file using the minimum value length (32755) which causes 
the bug to happen:

https://gist.github.com/wesm/56c66524b1237a08a528c6eb8d01e44b

The last seek before the exception is to the position 327853, but the file is 
only 459938 in length. Because 5 * 32755 is 163775, there's not enough bytes 
left in the file to decode the plain value page. It looks like the problem is 
in the Thrift decoding code which uses internal APIs of Thrift in Python:

{code}
2--> 32     file_obj.seek(starting_pos + blocks * bufsize + buffer_pos)
     33     return obj
     34 

ipdb> p starting_pos
229557
ipdb> p blocks
1
ipdb> p * bufsize
*** SyntaxError: invalid syntax
ipdb> p bufsize
65536
ipdb> p buffer_pos
32760
ipdb> p starting_pos + blocks * bufsize + buffer_pos
327853
ipdb> p starting_pos
229557
ipdb> p bufsize
65536
ipdb> p bt.cstringio_buf.tell()
32760
{code}

Whether the bug is in fastparquet or Thrift-Python is anyone's guess, but it 
appears the problem is caused by the fact that the metadata structure for the 
data page is large, because it contains statistics for 2x the size of the 
value, which is 65510. Including tags in the Thrift protocol, the size of the 
message is surely exceeding the buffer size of 65536 so I'm guessing that's the 
problem

Are other Parquet implementations able to read this file? If they are not I 
will be happy to keep investigating

> [Python] Can't read pyarrow string columns in fastparquet
> ---------------------------------------------------------
>
>                 Key: ARROW-3238
>                 URL: https://issues.apache.org/jira/browse/ARROW-3238
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: Python
>            Reporter: Theo Walker
>            Priority: Major
>              Labels: parquet
>
> Writing really long strings from pyarrow causes exception in fastparquet read.
> {code:java}
> Traceback (most recent call last):
> File "/Users/twalker/repos/cloud-atlas/diag/right.py", line 47, in <module>
> read_fastparquet()
> File "/Users/twalker/repos/cloud-atlas/diag/right.py", line 41, in 
> read_fastparquet
> dff = pf.to_pandas(['A'])
> File 
> "/Users/twalker/anaconda/lib/python2.7/site-packages/fastparquet/api.py", 
> line 426, in to_pandas
> index=index, assign=parts)
> File 
> "/Users/twalker/anaconda/lib/python2.7/site-packages/fastparquet/api.py", 
> line 258, in read_row_group
> scheme=self.file_scheme)
> File 
> "/Users/twalker/anaconda/lib/python2.7/site-packages/fastparquet/core.py", 
> line 344, in read_row_group
> cats, selfmade, assign=assign)
> File 
> "/Users/twalker/anaconda/lib/python2.7/site-packages/fastparquet/core.py", 
> line 321, in read_row_group_arrays
> catdef=out.get(name+'-catdef', None))
> File 
> "/Users/twalker/anaconda/lib/python2.7/site-packages/fastparquet/core.py", 
> line 235, in read_col
> skip_nulls, selfmade=selfmade)
> File 
> "/Users/twalker/anaconda/lib/python2.7/site-packages/fastparquet/core.py", 
> line 99, in read_data_page
> raw_bytes = _read_page(f, header, metadata)
> File 
> "/Users/twalker/anaconda/lib/python2.7/site-packages/fastparquet/core.py", 
> line 31, in _read_page
> page_header.uncompressed_page_size)
> AssertionError: found 175532 raw bytes (expected 200026){code}
> If written with compression, it reports compression errors instead:
> {code:java}
> SNAPPY: snappy.UncompressError: Error while decompressing: invalid input
> GZIP: zlib.error: Error -3 while decompressing data: incorrect header 
> check{code}
>  
>  
> Minimal code to reproduce:
> {code:java}
> import os
> import pandas as pd
> import pyarrow
> import pyarrow.parquet as arrow_pq
> from fastparquet import ParquetFile
> # data to generate
> ROW_LENGTH = 40000 # decreasing below 32750ish eliminates exception
> N_ROWS = 10
> # file write params
> ROW_GROUP_SIZE = 5 # Lower numbers eliminate exception, but strange data is 
> read (e.g. Nones)
> FILENAME = 'test.parquet'
> def write_arrow():
>     df = pd.DataFrame({'A': ['A'*ROW_LENGTH for _ in range(N_ROWS)]})
>     if os.path.isfile(FILENAME):
>         os.remove(FILENAME)
>     arrow_table = pyarrow.Table.from_pandas(df)
>     arrow_pq.write_table(arrow_table,
>     FILENAME,
>     use_dictionary=False,
>     compression='NONE',
>     row_group_size=ROW_GROUP_SIZE)
> def read_arrow():
>     print "arrow:"
>     table2 = arrow_pq.read_table(FILENAME)
>     print table2.to_pandas().head()
> def read_fastparquet():
>     print "fastparquet:"
>     pf = ParquetFile(FILENAME)
>     dff = pf.to_pandas(['A'])
>     print dff.head()
> write_arrow()
> read_arrow()
> read_fastparquet()
> {code}
> Versions:
> {code:java}
> fastparquet==0.1.6
> pyarrow==0.10.0
> pandas==0.22.0
> sys.version '2.7.15 |Anaconda custom (64-bit)| (default, May 1 2018, 
> 18:37:05) \n[GCC 4.2.1 Compatible Clang 4.0.1 (tags/RELEASE_401/final)]'{code}
> Also opened issue here: https://github.com/dask/fastparquet/issues/375



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

Reply via email to