-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
Comments
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.. |
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 |
A bug is found about reading parquet in python @alexeyche Would you mind provide some info about how to reproduce the issue here? |
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? |
@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. |
@jhwang7628 @alexeyche Would you mind check that whether you're using dictionary encoding for string/binary column? |
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) 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. |
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:
|
…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 <[email protected]> Co-authored-by: mwish <[email protected]> Co-authored-by: Gang Wu <[email protected]> Signed-off-by: Antoine Pitrou <[email protected]>
…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 <[email protected]> Co-authored-by: mwish <[email protected]> Co-authored-by: Gang Wu <[email protected]> Signed-off-by: Antoine Pitrou <[email protected]>
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! |
### 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 <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
…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 <[email protected]> Co-authored-by: mwish <[email protected]> Co-authored-by: Gang Wu <[email protected]> Signed-off-by: Antoine Pitrou <[email protected]>
This PR aims to upgrade Apache Arrow to 14.0.2. To use the latest bug fixed version in various environments - https://arrow.apache.org/release/14.0.2.html - apache/arrow#38577 No. Pass the CIs. No. Closes apache#44472 from dongjoon-hyun/SPARK-46496. Authored-by: Dongjoon Hyun <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
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:Is this an intended behavior? I skimmed through the changelog but did not find this. Thanks.
Component(s)
Python
The text was updated successfully, but these errors were encountered: