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
da.store loses dependency information #8380
Comments
Also note this does not happen if the map_blocks function is a single result or if I don't do |
Ok new find, if I call |
Found an ugly workaround, change the bottom portion of the script to turn fuse.active off: with dask.config.set(schedulers='single-threaded'):
with dask.config.set({"optimization.fuse.active": False}):
store_res1 = da.store(src[0], target1, compute=False)
store_res2 = da.store(src[1], target2, compute=False)
da.compute(store_res1, store_res2)
# one call per chunk
assert TOTAL_CALLS == start.blocks.size This stops |
It seems to me that if optimization should not happen until right before computation starts. So in the |
Correct, |
Would that just mean putting an Lines 1017 to 1019 in f588189
|
without optimization, you need to merge sources_hlg into the final graph - see HighLevelGraph.merge |
I'm not familiar with the structural differences between the various dictionaries and graphs used here, but the line just above that uses Lines 1016 to 1022 in f588189
So theoretically, if we're not optimizing then we can make |
shallow copy, but yes. You'll also need to find the topmost layers using |
@crusaderky I'm finally making time to fix this, but I'm still having trouble with some of the HLG graph interfaces. The main question I'm coming up with now is: is there any reason to optimize the graphs at all in |
@djhoese no I don't think the graph should be optimized. |
@crusaderky I made #9732 so let me know what you think. What I was trying to say is that there is no need to optimize the graph explicitly regardless of the |
What happened:
I'm trying to take the result from a
map_blocks
function and store one slice of the resulting array in one zarr array and another slice in another array. I setcompute=False
on theda.store
calls so that I can compute them together later and avoid computing the array multiple times. However, it seems the map blocks function is called for each store operation. From what I can tell the dependency graph is getting lost.What you expected to happen:
Two
da.store
calls with shared tasks on their dask graphs should share computations when computed at the same time.Minimal Complete Verifiable Example:
Anything else we need to know?:
As mentioned above, I'm actually trying to use
to_zarr
to save some dask arrays. I want to write them to zarr and then get the resulting loaded zarr arrays as results, but I couldn't find a way to do that with the combination ofreturn_stored
andcompute=False
as you need to compute to write to the zarr arrays, but that then returns the resulting numpy arrays.Also note that my map_blocks function is returning the
np.stack
because it is actually returning two results.Environment:
The text was updated successfully, but these errors were encountered: