-
Notifications
You must be signed in to change notification settings - Fork 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
[Dataset] ray failed to serialize pyarrow7.0.0 Tables #22310
Comments
Hmm interesting, @scv119 I tried to reproduce this locally and got ~the same high-water mark for object store utilization and number of bytes consumed by tasks when comparing |
Here's a minimal repro. With pyarrow==6.0.1 it works fine. With 7.0, you get a 400GiB plasma allocated block...
The issue seems to happen when putting an Arrow table read from parquet into plasma. |
Inserting a copy() of the block seems to resolve the issue. Maybe there's some large sparse array that's allocated in the pyarrow object when read from parquet in pyarrow 7.0.0. We can insert the copy as a workaround for that version, and file a bug upstream. |
Interesting, I tried reading Parquet data and many different operation combinations and wasn't able to reproduce.
I'll time-box an investigation this week and otherwise do as you suggest. Thank you for finding a repro! |
After some debugging, it looks like this is the same bug around serializing Arrow array slice-views on an underlying larger buffer. The new issue here is that Arrow's Parquet reader is creating chunked arrays whose chunks are each slice-views on an underlying contiguous buffer containing the entire chunked array, so when the chunked array is serialized, serializing the Here is a minimal reproduction without involving Parquet (this is buggy in all Arrow versions):
The change in Arrow 7.0.0 is that the chunked array created for a given column when reading Parquet files is all pointing at a single contiguous buffer; in past versions, a different backing buffer was created for each chunk.
Here you can see that Arrow 6.0.1 created a different backing buffer for each chunk:
Possible Solutions
I'd vote for pushing on (2) and (3) in parallel, with (2) being a short-term solution that should transparently apply to all cases, and (3) being the long-term fix. |
Great find. For (2), I think that sounds good if the implementation is fairly simple. Otherwise, we can add a smaller fix with the read logic. For (3) that also sounds good; we should at least ping that this is impacting downstream users. |
Search before asking
Ray Component
Ray Core
What happened + What you expected to happen
See #22253 (comment) and #22177 for more context.
TL;DR, when serialize pyarrow7.0.0 Tables, the serialization has wrong data size (as big as 200GBs).
Versions / Dependencies
pyarrow 7.0.0
Reproduction script
python ray/release/nightly_tests/dataset/ray_sgd_runner.py --num-epochs 1 --smoke-test
with pyarrow 7.0.0 installed
Anything else
No response
Are you willing to submit a PR?
The text was updated successfully, but these errors were encountered: