Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
0.15.0
Description
Columns of type array<primitive type> (such as `array<int32>`, `array<int64>`...) are not readable anymore using pyarrow == 0.15.0 (but were with pyarrow == 0.14.1) when the original writer of the parquet file is parquet-mr 1.9.1.
import pyarrow.parquet as pq pf = pq.ParquetFile('sample.gz.parquet') print(pf.read(columns=['profile_ids']))
with 0.14.1:
pyarrow.Table profile_ids: list<element: int64> child 0, element: int64 ...
with 0.15.0:
Traceback (most recent call last): File "<string>", line 1, in <module> File "/Library/Frameworks/Python.framework/Versions/3.6/lib/python3.6/site-packages/pyarrow/parquet.py", line 253, in read use_threads=use_threads) File "pyarrow/_parquet.pyx", line 1131, in pyarrow._parquet.ParquetReader.read_all File "pyarrow/error.pxi", line 78, in pyarrow.lib.check_status pyarrow.lib.ArrowInvalid: Column data for field 0 with type list<item: int64> is inconsistent with schema list<element: int64>
I've tested parquet files coming from multiple tables (with various schemas) created with `parquet-mr`, couldn't read any `array<primitive type>` column anymore.
I think the bug was introduced with [this commit|https://github.com/apache/arrow/commit/06fd2da5e8e71b660e6eea4b7702ca175e31f3f5].
I think the root of the issue comes from the fact that `parquet-mr` writes the inner struct name as `"element"` by default (see here), whereas `parquet-cpp` (or `pyarrow`?) assumes `"item"` (see for example this test). The round-tripping tests write/read in pyarrow only obviously won't catch this.
Attachments
Attachments
Issue Links
- duplicates
-
ARROW-6719 Parquet read_table error in Python3.7: pyarrow.lib.ArrowInvalid: Column data for field with type list<...> is inconsistent with schema list<...>
- Closed
-
ARROW-6849 [Python] can not read a parquet store containing a list of integers
- Closed
- links to