[ https://issues.apache.org/jira/browse/PARQUET-1857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17100081#comment-17100081 ]
Novice edited comment on PARQUET-1857 at 5/5/20, 5:04 PM: ---------------------------------------------------------- Thanks Wes. So I tried to reduce the number of row groups, to 41. Here is the error I got: Pyarrow: ``` >>> df = pd.read_parquet("test.parquet", engine="pyarrow") Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line 296, in read_parquet return impl.read(path, columns=columns, **kwargs) File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line 125, in read path, columns=columns, **kwargs File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", line 1281, in read_table use_pandas_metadata=use_pandas_metadata) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", line 1137, in read use_pandas_metadata=use_pandas_metadata) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", line 605, in read table = reader.read(**options) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", line 253, in read use_threads=use_threads) File "pyarrow/_parquet.pyx", line 1136, in pyarrow._parquet.ParquetReader.read_all File "pyarrow/error.pxi", line 99, in pyarrow.lib.check_status OSError: Unexpected end of stream ``` fastparquet: ``` >>> df = pd.read_parquet("test.parquet", engine="fastparquet") /home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/encoding.py:222: NumbaDeprecationWarning: The 'numba.jitclass' decorator has moved to 'numba.experimental.jitclass' to better reflect the experimental nature of the functionality. Please update your imports to accommodate this change and see [http://numba.pydata.org/numba-doc/latest/reference/deprecation.html#change-of-jitclass-location] for the time frame. Numpy8 = numba.jitclass(spec8)(NumpyIO) /home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/encoding.py:224: NumbaDeprecationWarning: The 'numba.jitclass' decorator has moved to 'numba.experimental.jitclass' to better reflect the experimental nature of the functionality. Please update your imports to accommodate this change and see [http://numba.pydata.org/numba-doc/latest/reference/deprecation.html#change-of-jitclass-location] for the time frame. Numpy32 = numba.jitclass(spec32)(NumpyIO) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line 296, in read_parquet return impl.read(path, columns=columns, **kwargs) File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line 201, in read return parquet_file.to_pandas(columns=columns, **kwargs) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/api.py", line 399, in to_pandas index=index, assign=parts) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/api.py", line 228, in read_row_group scheme=self.file_scheme) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 354, in read_row_group cats, selfmade, assign=assign) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 331, in read_row_group_arrays catdef=out.get(name+'-catdef', None)) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 245, in read_col skip_nulls, selfmade=selfmade) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 99, in read_data_page raw_bytes = _read_page(f, header, metadata) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 31, in _read_page page_header.uncompressed_page_size) AssertionError: found 120016208 raw bytes (expected None) ``` The corresponding Rust code is: ``` use parquet::{ column::writer::ColumnWriter::BoolColumnWriter, column::writer::ColumnWriter::Int32ColumnWriter, [file::] { properties::WriterProperties, writer::\\{FileWriter, SerializedFileWriter} , }, schema::parser::parse_message_type, }; use std::\{fs, rc::Rc}; fn main() { let schema = " message schema { REQUIRED INT32 a; REQUIRED BOOLEAN b; } "; let schema = Rc::new(parse_message_type(schema).unwrap()); let props = Rc::new( WriterProperties::builder() .set_statistics_enabled(false) .set_dictionary_enabled(false) .build(), ); let file = fs::File::create("test.parquet").unwrap(); let mut writer = SerializedFileWriter::new(file, schema, props).unwrap(); let batch_size = 1_000_000; let mut data = vec![]; let mut data_bool = vec![]; for i in 0..batch_size { data.push(i); data_bool.push(true); } let mut j = 0; loop { let mut row_group_writer = writer.next_row_group().unwrap(); let mut col_writer = row_group_writer.next_column().unwrap().unwrap(); if let Int32ColumnWriter(ref mut typed_writer) = col_writer { typed_writer.write_batch(&data, None, None).unwrap(); } else { panic!(); } row_group_writer.close_column(col_writer).unwrap(); let mut col_writer = row_group_writer.next_column().unwrap().unwrap(); if let BoolColumnWriter(ref mut typed_writer) = col_writer \{ typed_writer.write_batch(&data_bool, None, None).unwrap(); } else \{ panic!(); } row_group_writer.close_column(col_writer).unwrap(); writer.close_row_group(row_group_writer).unwrap(); j += 1; if j * batch_size > 40_000_000 { break; } } writer.close().unwrap() } ``` Please see the test_2.parquet.tgz for more details. was (Author: novice): Thanks Wes. So I tried to reduce the number of row groups, to 41. Here is the error I got: ``` >>> df = pd.read_parquet("test.parquet", engine="pyarrow") Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line 296, in read_parquet return impl.read(path, columns=columns, **kwargs) File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line 125, in read path, columns=columns, **kwargs File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", line 1281, in read_table use_pandas_metadata=use_pandas_metadata) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", line 1137, in read use_pandas_metadata=use_pandas_metadata) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", line 605, in read table = reader.read(**options) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", line 253, in read use_threads=use_threads) File "pyarrow/_parquet.pyx", line 1136, in pyarrow._parquet.ParquetReader.read_all File "pyarrow/error.pxi", line 99, in pyarrow.lib.check_status OSError: Unexpected end of stream >>> df = pd.read_parquet("test.parquet", engine="fastparquet") /home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/encoding.py:222: NumbaDeprecationWarning: The 'numba.jitclass' decorator has moved to 'numba.experimental.jitclass' to better reflect the experimental nature of the functionality. Please update your imports to accommodate this change and see http://numba.pydata.org/numba-doc/latest/reference/deprecation.html#change-of-jitclass-location for the time frame. Numpy8 = numba.jitclass(spec8)(NumpyIO) /home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/encoding.py:224: NumbaDeprecationWarning: The 'numba.jitclass' decorator has moved to 'numba.experimental.jitclass' to better reflect the experimental nature of the functionality. Please update your imports to accommodate this change and see http://numba.pydata.org/numba-doc/latest/reference/deprecation.html#change-of-jitclass-location for the time frame. Numpy32 = numba.jitclass(spec32)(NumpyIO) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line 296, in read_parquet return impl.read(path, columns=columns, **kwargs) File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line 201, in read return parquet_file.to_pandas(columns=columns, **kwargs) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/api.py", line 399, in to_pandas index=index, assign=parts) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/api.py", line 228, in read_row_group scheme=self.file_scheme) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 354, in read_row_group cats, selfmade, assign=assign) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 331, in read_row_group_arrays catdef=out.get(name+'-catdef', None)) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 245, in read_col skip_nulls, selfmade=selfmade) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 99, in read_data_page raw_bytes = _read_page(f, header, metadata) File "/home/miniconda3/envs/ds/lib/python3.7/site-packages/fastparquet/core.py", line 31, in _read_page page_header.uncompressed_page_size) AssertionError: found 120016208 raw bytes (expected None) ``` The corresponding Rust code is: ``` use parquet::{ column::writer::ColumnWriter::BoolColumnWriter, column::writer::ColumnWriter::Int32ColumnWriter, file::{ properties::WriterProperties, writer::\{FileWriter, SerializedFileWriter}, }, schema::parser::parse_message_type, }; use std::\{fs, rc::Rc}; fn main() { let schema = " message schema { REQUIRED INT32 a; REQUIRED BOOLEAN b; } "; let schema = Rc::new(parse_message_type(schema).unwrap()); let props = Rc::new( WriterProperties::builder() .set_statistics_enabled(false) .set_dictionary_enabled(false) .build(), ); let file = fs::File::create("test.parquet").unwrap(); let mut writer = SerializedFileWriter::new(file, schema, props).unwrap(); let batch_size = 1_000_000; let mut data = vec![]; let mut data_bool = vec![]; for i in 0..batch_size { data.push(i); data_bool.push(true); } let mut j = 0; loop { let mut row_group_writer = writer.next_row_group().unwrap(); let mut col_writer = row_group_writer.next_column().unwrap().unwrap(); if let Int32ColumnWriter(ref mut typed_writer) = col_writer { typed_writer.write_batch(&data, None, None).unwrap(); } else { panic!(); } row_group_writer.close_column(col_writer).unwrap(); let mut col_writer = row_group_writer.next_column().unwrap().unwrap(); if let BoolColumnWriter(ref mut typed_writer) = col_writer { typed_writer.write_batch(&data_bool, None, None).unwrap(); } else { panic!(); } row_group_writer.close_column(col_writer).unwrap(); writer.close_row_group(row_group_writer).unwrap(); j += 1; if j * batch_size > 40_000_000 { break; } } writer.close().unwrap() } ``` Please see the test_2.parquet.tgz for more details. > [C++][Parquet] ParquetFileReader unable to read files with more than 32767 > row groups > ------------------------------------------------------------------------------------- > > Key: PARQUET-1857 > URL: https://issues.apache.org/jira/browse/PARQUET-1857 > Project: Parquet > Issue Type: Bug > Components: parquet-cpp > Reporter: Novice > Assignee: Wes McKinney > Priority: Critical > Labels: pull-request-available > Attachments: test.parquet.tgz, test_2.parquet.tgz > > Time Spent: 20m > Remaining Estimate: 0h > > I am using Rust to write Parquet file and read from Python. > When write_batch with 10000 batch size, reading the Parquet file from Python > gives the error below: > ``` > >>> pd.read_parquet("some.parquet", engine="pyarrow") > Traceback (most recent call last): > File "<stdin>", line 1, in <module> > File "/home//.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 296, in read_parquet > return impl.read(path, columns=columns, **kwargs) > File "/home//.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 125, in read > path, columns=columns, **kwargs > File > "/home//miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1537, in read_table > use_pandas_metadata=use_pandas_metadata) > File > "/home//miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1262, in read > use_pandas_metadata=use_pandas_metadata) > File > "/home//miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 707, in read > table = reader.read(**options) > File > "/home//miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 337, in read > use_threads=use_threads) > File "pyarrow/_parquet.pyx", line 1130, in > pyarrow._parquet.ParquetReader.read_all > File "pyarrow/error.pxi", line 100, in pyarrow.lib.check_status > OSError: Unexpected end of stream > ``` > Also, when using batch size 1 and then read from Python, there is error too: > ``` > >>> pd.read_parquet("some.parquet", engine="pyarrow") > Traceback (most recent call last): > File "<stdin>", line 1, in <module> > File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 296, in read_parquet > return impl.read(path, columns=columns, **kwargs) > File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 125, in read > path, columns=columns, **kwargs > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1537, in read_table > use_pandas_metadata=use_pandas_metadata) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1262, in read > use_pandas_metadata=use_pandas_metadata) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 707, in read > table = reader.read(**options) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 337, in read > use_threads=use_threads) > File "pyarrow/_parquet.pyx", line 1130, in > pyarrow._parquet.ParquetReader.read_all > File "pyarrow/error.pxi", line 100, in pyarrow.lib.check_status > OSError: The file only has 0 columns, requested metadata for column: 6 > ``` > Using batch size 1000 is fine. > Note that my data has 450047 rows. Schema: > ``` > message schema > { REQUIRED INT32 a; REQUIRED INT32 b; REQUIRED INT32 c; REQUIRED INT64 d; > REQUIRED INT32 e; REQUIRED BYTE_ARRAY f (UTF8); REQUIRED BOOLEAN g; } > ``` > > EDIT: as I add more rows (estimated 80 millions), using batch size 1000 does > not work too: > ``` > >>> df = pd.read_parquet("data/ping_pong.parquet", engine="pyarrow") > Traceback (most recent call last): > File "<stdin>", line 1, in <module> > File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 296, in read_parquet > return impl.read(path, columns=columns, **kwargs) > File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 125, in read > path, columns=columns, **kwargs > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1537, in read_table > use_pandas_metadata=use_pandas_metadata) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1262, in read > use_pandas_metadata=use_pandas_metadata) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 707, in read > table = reader.read(**options) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 337, in read > use_threads=use_threads) > File "pyarrow/_parquet.pyx", line 1130, in > pyarrow._parquet.ParquetReader.read_all > File "pyarrow/error.pxi", line 100, in pyarrow.lib.check_status > OSError: The file only has 0 columns, requested metadata for column: 6 > ``` > Unless I am using it wrong (which doesn't seem to be, since the API is > simple), this is not usable at all :( > > EDIT: some more logs, using 1000 batch size, a lot of rows: > ``` > >>> df = pd.read_parquet("ping_pong.parquet", engine="pyarrow") > Traceback (most recent call last): > File "<stdin>", line 1, in <module> > File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 296, in read_parquet > return impl.read(path, columns=columns, **kwargs) > File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 125, in read > path, columns=columns, **kwargs > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1537, in read_table > use_pandas_metadata=use_pandas_metadata) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1262, in read > use_pandas_metadata=use_pandas_metadata) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 707, in read > table = reader.read(**options) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 337, in read > use_threads=use_threads) > File "pyarrow/_parquet.pyx", line 1130, in > pyarrow._parquet.ParquetReader.read_all > File "pyarrow/error.pxi", line 100, in pyarrow.lib.check_status > OSError: The file only has -959432807 columns, requested metadata for > column: 6 > ``` > > EDIT: > I wanted to try fastparquet, but seems fastparquet does not support > .set_dictionary_enabled(true), so I set it to false. > Turns out fastparquet is fine, so likely a problem with pyarrow. > ``` > >>> df = pd.read_parquet("data/ping_pong.parquet", engine="pyarrow") > Traceback (most recent call last): > File "<stdin>", line 1, in <module> > File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 296, in read_parquet > return impl.read(path, columns=columns, **kwargs) > File "/home/.local/lib/python3.7/site-packages/pandas/io/parquet.py", line > 125, in read > path, columns=columns, **kwargs > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1281, in read_table > use_pandas_metadata=use_pandas_metadata) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 1137, in read > use_pandas_metadata=use_pandas_metadata) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 605, in read > table = reader.read(**options) > File > "/home/miniconda3/envs/ds/lib/python3.7/site-packages/pyarrow/parquet.py", > line 253, in read > use_threads=use_threads) > File "pyarrow/_parquet.pyx", line 1136, in > pyarrow._parquet.ParquetReader.read_all > File "pyarrow/error.pxi", line 99, in pyarrow.lib.check_status > OSError: The file only has -580697109 columns, requested metadata for > column: 5 > >>> df = pd.read_parquet("data/ping_pong.parquet", engine="fastparquet") > ``` -- This message was sent by Atlassian Jira (v8.3.4#803005)