Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Reading parquet file behavior change from 13.0.0 to 14.0.0 #38577

Closed
jhwang7628 opened this issue Nov 3, 2023 · 9 comments · Fixed by #38784
Closed

Reading parquet file behavior change from 13.0.0 to 14.0.0 #38577

jhwang7628 opened this issue Nov 3, 2023 · 9 comments · Fixed by #38784

Comments

@jhwang7628
Copy link

Describe the bug, including details regarding any error messages, version, and platform.

Hi,

We have a parquet that used to read fine in 13.0.0, but now I got an error when calling via pandas.read_parquet using 14.0.0. The relevant error is:

  File "/opt/venv/lib/python3.10/site-packages/pyarrow/parquet/core.py", line 3003, in read_table
    return dataset.read(columns=columns, use_threads=use_threads,   
  File "/opt/venv/lib/python3.10/site-packages/pyarrow/parquet/core.py", line 2631, in read
    table = self._dataset.to_table(  
  File "pyarrow/_dataset.pyx", line 556, in pyarrow._dataset.Dataset.to_table
  File "pyarrow/_dataset.pyx", line 3713, in pyarrow._dataset.Scanner.to_table
  File "pyarrow/error.pxi", line 154, in pyarrow.lib.pyarrow_internal_check_status
  File "pyarrow/error.pxi", line 91, in pyarrow.lib.check_status
pyarrow.lib.ArrowCapacityError: array cannot contain more than 2147483646 bytes, have 2148480400

Is this an intended behavior? I skimmed through the changelog but did not find this. Thanks.

Component(s)

Python

@mapleFU
Copy link
Member

mapleFU commented Nov 4, 2023

Hmmm would you mind provide the file or a minimal reproduce way? A bit hard to check the scanner change without the data or logging..

@alexeyche
Copy link

Also noticed this, though hard to reproduce it without sharing data as of now, I might find time in future. In short it's just big dataset with nested arrays of floats

@mapleFU
Copy link
Member

mapleFU commented Nov 7, 2023

#38621

A bug is found about reading parquet in python read_table calls more requests than expected. But I don't now if this is related to the issue.

@alexeyche Would you mind provide some info about how to reproduce the issue here?

@jhwang7628
Copy link
Author

I cannot provide the data blob as it is company internal data. The issue is consistent and deterministic on my side, so I think maybe any large parquet will do?

@mapleFU
Copy link
Member

mapleFU commented Nov 8, 2023

@jhwang7628 After go though the code, I think this might related to #38437

I'll try to check this in this week. This might trable when you read a large binary column, after this patch it might reserve more data than expected.

@mapleFU
Copy link
Member

mapleFU commented Nov 19, 2023

@jhwang7628 @alexeyche Would you mind check that whether you're using dictionary encoding for string/binary column?

@jorisvandenbossche
Copy link
Member

jorisvandenbossche commented Nov 22, 2023

I cannot provide the data blob as it is company internal data. The issue is consistent and deterministic on my side, so I think maybe any large parquet will do?

Would you be able to try to create a file that has the same characteristics as your internal data file but with random data? (eg approximately the same types, and number of rows and size)
That would help a lot in evaluating if we are actually fixing this issue with #38784

Or otherwise, giving some more information about the data types being used, the encoding and row group sizes in the Parquet file, etc could also help to reproduce it on our side.

@jorisvandenbossche
Copy link
Member

From seeing the potential fix in #38784, I could manage to create a simple reproducer:

Creating this file with pyarrow 13.0 reads fine with that version:

import string
import numpy as np
import pyarrow as pa

# column with >2GB data
data = ["".join(np.random.choice(list(string.ascii_letters), n)) for n in np.random.randint(10, 500, size=10_000)]
table = pa.table({'a': pa.array(data*1000)})

import pyarrow.parquet as pq
pq.write_table(table, "test_capacity.parquet")

but reading with pyarrow 14:

import pyarrow.parquet as pq
pf = pq.ParquetFile("test_capacity.parquet")

In [6]: pf.read()
...
ArrowCapacityError: array cannot contain more than 2147483646 bytes, have 2148282365
/home/joris/scipy/repos/arrow/cpp/src/arrow/array/builder_binary.h:332  ValidateOverflow(elements)
/home/joris/scipy/repos/arrow/cpp/src/parquet/encoding.cc:1202  acc_->builder->ReserveData( std::min<int64_t>(*estimated_data_length, ::arrow::kBinaryMemoryLimit))
/home/joris/scipy/repos/arrow/cpp/src/parquet/encoding.cc:1407  helper.Prepare(len_)
/home/joris/scipy/repos/arrow/cpp/src/parquet/arrow/reader.cc:109  LoadBatch(batch_size)
/home/joris/scipy/repos/arrow/cpp/src/parquet/arrow/reader.cc:1252  ReadColumn(static_cast<int>(i), row_groups, reader.get(), &column)
/home/joris/scipy/repos/arrow/cpp/src/parquet/arrow/reader.cc:1233  fut.MoveResult()

@AlenkaF AlenkaF added this to the 15.0.0 milestone Nov 27, 2023
pitrou pushed a commit that referenced this issue Nov 27, 2023
…tByteArrayDecoderImpl (#38784)

### Rationale for this change

Do some changes mentioned in #38432

I believe this might fix #38577

Problem1:

The `BinaryHelper` might call `Prepare()` and `Prepare(estimated-output-binary-length)` for data. This might because:

1. For Plain Encoding ByteArray, the `len_` is similar to the data-page size, so `Reserve` is related.
2. For Dict Encoding. The Data Page is just a RLE encoding Page, it's `len_` might didn't directly related to output-binary. 

Problem2:

`Prepare` using `::arrow::kBinaryMemoryLimit` as min-value, we should use `this->chunk_space_remaining_`.

Problem3:

`std::optional<int64_t>` is hard to optimize for some compilers

### What changes are included in this PR?

Mention the behavior of BinaryHelper. And trying to fix it.

### Are these changes tested?

No

### Are there any user-facing changes?

Regression fixes

* Closes: #38432

Lead-authored-by: mwish <maplewish117@gmail.com>
Co-authored-by: mwish <1506118561@qq.com>
Co-authored-by: Gang Wu <ustcwg@gmail.com>
Signed-off-by: Antoine Pitrou <antoine@python.org>
@raulcd raulcd modified the milestones: 15.0.0, 14.0.2 Nov 28, 2023
raulcd pushed a commit that referenced this issue Nov 28, 2023
…tByteArrayDecoderImpl (#38784)

### Rationale for this change

Do some changes mentioned in #38432

I believe this might fix #38577

Problem1:

The `BinaryHelper` might call `Prepare()` and `Prepare(estimated-output-binary-length)` for data. This might because:

1. For Plain Encoding ByteArray, the `len_` is similar to the data-page size, so `Reserve` is related.
2. For Dict Encoding. The Data Page is just a RLE encoding Page, it's `len_` might didn't directly related to output-binary. 

Problem2:

`Prepare` using `::arrow::kBinaryMemoryLimit` as min-value, we should use `this->chunk_space_remaining_`.

Problem3:

`std::optional<int64_t>` is hard to optimize for some compilers

### What changes are included in this PR?

Mention the behavior of BinaryHelper. And trying to fix it.

### Are these changes tested?

No

### Are there any user-facing changes?

Regression fixes

* Closes: #38432

Lead-authored-by: mwish <maplewish117@gmail.com>
Co-authored-by: mwish <1506118561@qq.com>
Co-authored-by: Gang Wu <ustcwg@gmail.com>
Signed-off-by: Antoine Pitrou <antoine@python.org>
@jhwang7628
Copy link
Author

jhwang7628 commented Nov 29, 2023

Thanks! I missed the above comments. Glad you were able to repro yourself. I'll wait until this PR gets released to re-test it on my data. Thanks again!

dongjoon-hyun added a commit to apache/spark that referenced this issue Dec 25, 2023
### What changes were proposed in this pull request?

This PR aims to upgrade Apache Arrow to 14.0.2.

### Why are the changes needed?

To use the latest bug fixed version in various environments
- https://arrow.apache.org/release/14.0.2.html
  - apache/arrow#38577

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the CIs.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44472 from dongjoon-hyun/SPARK-46496.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
dgreiss pushed a commit to dgreiss/arrow that referenced this issue Feb 19, 2024
…he DictByteArrayDecoderImpl (apache#38784)

### Rationale for this change

Do some changes mentioned in apache#38432

I believe this might fix apache#38577

Problem1:

The `BinaryHelper` might call `Prepare()` and `Prepare(estimated-output-binary-length)` for data. This might because:

1. For Plain Encoding ByteArray, the `len_` is similar to the data-page size, so `Reserve` is related.
2. For Dict Encoding. The Data Page is just a RLE encoding Page, it's `len_` might didn't directly related to output-binary. 

Problem2:

`Prepare` using `::arrow::kBinaryMemoryLimit` as min-value, we should use `this->chunk_space_remaining_`.

Problem3:

`std::optional<int64_t>` is hard to optimize for some compilers

### What changes are included in this PR?

Mention the behavior of BinaryHelper. And trying to fix it.

### Are these changes tested?

No

### Are there any user-facing changes?

Regression fixes

* Closes: apache#38432

Lead-authored-by: mwish <maplewish117@gmail.com>
Co-authored-by: mwish <1506118561@qq.com>
Co-authored-by: Gang Wu <ustcwg@gmail.com>
Signed-off-by: Antoine Pitrou <antoine@python.org>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
6 participants