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

[Datasets] Arrow data buffers aren't truncated when pickling zero-copy slice views, leading to huge serialization bloat #29814

Closed
clarkzinzow opened this issue Oct 28, 2022 · 2 comments · Fixed by #29993
Assignees
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P1 Issue that should be fixed within a few weeks

Comments

@clarkzinzow
Copy link
Contributor

This is a duplicating issue of #22310, so we have a clean issue on which to summarize the problem.

When pickling Arrow data (tables, in our case), Arrow currently doesn't truncate the data buffers for zero-copy slice views of the data (see Arrow ticket). For many of our workloads, this results is sending a potentially very large data buffer over the network for a very small slice. For pathological cases like shuffling, where we might chunk a data block into 1000 chunks and send each to a different reduce, this would involve sending 1000 copies of the entire data block; for a 1 GiB data block, we'd send an aggregate 1 TiB of data over the network instead of the expected 1 GiB. This makes Ray + Arrow essentially unusable.

This bug exists in Arrow 6 through Arrow 10; we worked around the bug in Arrow 6 via explicit copying when slicing data, e.g. in our shuffle implementation, but Arrow 7+ adds internal zero-copy slicing that makes this workaround coverage untenable (e.g. when reading Parquet).

Solution

We can bypass this buggy pickle path by registering a custom serializer for Arrow data that properly truncates these buffers.

@clarkzinzow clarkzinzow added bug Something that is supposed to be working; but isn't P1 Issue that should be fixed within a few weeks air data Ray Data-related issues labels Oct 28, 2022
@clarkzinzow clarkzinzow added this to the Arrow 7+ Support milestone Oct 28, 2022
@clarkzinzow clarkzinzow self-assigned this Oct 28, 2022
@pcmoritz
Copy link
Contributor

pcmoritz commented Nov 2, 2022

Is it possible to fix the pickling path upstream and truncate the buffers there?

@clarkzinzow
Copy link
Contributor Author

@pcmoritz Yes, I'm planning on doing that for Arrow 11: https://issues.apache.org/jira/browse/ARROW-10739

clarkzinzow added a commit that referenced this issue Nov 8, 2022
… Arrow serialization bug. (#29993)

This PR adds support for Arrow 7 in Ray, and is the second PR in a set of stacked PRs making up this mono-PR for Arrow 7+ support: #29161, and is stacked on top of a PR fixing task cancellation in Ray Core: #29984.

This PR:
- fixes a serialization bug in Arrow with a custom serializer for Arrow data ([Datasets] Arrow data buffers aren't truncated when pickling zero-copy slice views, leading to huge serialization bloat #29814)
- removes a bunch of defensive copying of Arrow data, which was a workaround for the aforementioned Arrow serialization bug
- adds a CI job for Arrow 7
- bumps the pyarrow upper bound to 8.0.0
WeichenXu123 pushed a commit to WeichenXu123/ray that referenced this issue Dec 19, 2022
… Arrow serialization bug. (ray-project#29993)

This PR adds support for Arrow 7 in Ray, and is the second PR in a set of stacked PRs making up this mono-PR for Arrow 7+ support: ray-project#29161, and is stacked on top of a PR fixing task cancellation in Ray Core: ray-project#29984.

This PR:
- fixes a serialization bug in Arrow with a custom serializer for Arrow data ([Datasets] Arrow data buffers aren't truncated when pickling zero-copy slice views, leading to huge serialization bloat ray-project#29814)
- removes a bunch of defensive copying of Arrow data, which was a workaround for the aforementioned Arrow serialization bug
- adds a CI job for Arrow 7
- bumps the pyarrow upper bound to 8.0.0

Signed-off-by: Weichen Xu <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't data Ray Data-related issues P1 Issue that should be fixed within a few weeks
Projects
None yet
3 participants