-
Notifications
You must be signed in to change notification settings - Fork 19
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
dask.array writing with distributed / multiprocessing schedulers #185
Comments
I brought up the fact that @shoyer suggested marking the store as read-only during unpickling to loudly fail in this scenario. That would allow thread-only parallelism and fail for anything else. I like this idea, with one modification. We can have the user explicitly opt-in to receiving a writeable store after pickling. with store.enable_distributed_writes():
do_smart_things That way they know they need to be careful. In our own The following will still be a footgun with store.enable_distributed_writes():
ds.to_zarr(store, ...) but we can have the error message suggest using Thoughts? |
I think this is a very reasonable approach Deepak. 👍 to what you have proposed above. |
Icechunk does support distributed writing of Arrays. However, currently Icechunk does not allow writing arrays via
dask.array.store
with distributed or multiprocessing schedulers.This is because Icechunk must gather the results of each distributed write operation (metadata about the chunks that were written) back to the client in order to commit the transaction. Without this step, each worker's writes would be lost.
How it works today with regular Zarr
Once each worker has written their piece of data to storage, it's done.
Instead with Icechunk, it needs to look like this
The dask.array.store code does not allow for returning metadata back to from each write process:
https://github.com/dask/dask/blob/20eeeda610260287a0dd50e4dd7b6a3cd8e007f3/dask/array/core.py#L1067
To overcome this, we need to either:
This does work with the dask threaded scheduler because the same icechunk store can be shared in memory between threads.
There are some parallels here to what is required to allow Dask to write iceberg tables (apache/iceberg#5800).
The text was updated successfully, but these errors were encountered: