-
-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Closed
Labels
Description
Looks like I caught our first severe regression introduced by #10535
import xarray as xr
import dask.array as da
from dask.base import visualize
from dask.order import diagnostics
from dask.base import collections_to_dsk
import flox.xarray
dummy = xr.Dataset(
{"foo": (("x", "y", "z"), da.ones((3, 10, 10), chunks=(1, 5, 5)))},
{"x": ["a", "b", "c"], "y": range(10)},
)
result = flox.xarray.xarray_reduce(
dummy,
dummy.x,
func="sum",
)
print(max(diagnostics(collections_to_dsk([result]))[1]))
visualize(
result,
filename='flox-reduction-order.png',
color='order',
optimize_graph=True,
)
Pre #10535
max_pressure: 6
Tasks are loaded as required and reducers are scheduled promptly. This is nice.
Main (w/ regression)
We can see that dependents of root tasks 1
and 2
are scheduled greedily. This can become quite bad for very large graphs.
Raw graph for testing
a, b, c, d, e = abcde
dsk = {
(a, 0): 0,
(a, 1): 0,
(a, 2): 0,
(b, 0, 0, 0): (f, (a, 0), (1, 5, 5)),
(b, 0, 0, 1): (f, (a, 1), (1, 5, 5)),
(b, 0, 0, 2): (f, (a, 2), (1, 5, 5)),
(b, 0, 1, 0): (f, (a, 0), (1, 5, 5)),
(b, 0, 1, 1): (f, (a, 1), (1, 5, 5)),
(b, 0, 1, 2): (f, (a, 2), (1, 5, 5)),
(b, 1, 0, 0): (f, (a, 0), (1, 5, 5)),
(b, 1, 0, 1): (f, (a, 1), (1, 5, 5)),
(b, 1, 0, 2): (f, (a, 2), (1, 5, 5)),
(b, 1, 1, 0): (f, (a, 0), (1, 5, 5)),
(b, 1, 1, 1): (f, (a, 1), (1, 5, 5)),
(b, 1, 1, 2): (f, (a, 2), (1, 5, 5)),
(c, 0, 0, 1): (f, [(b, 0, 1, 0), (b, 0, 1, 1), (b, 0, 1, 2)]),
(c, 0, 1, 0): (f, [(b, 1, 0, 0), (b, 1, 0, 1), (b, 1, 0, 2)]),
(c, 0, 1, 1): (f, [(b, 1, 1, 0), (b, 1, 1, 1), (b, 1, 1, 2)]),
(c, 0, 0, 0): (f, [(b, 0, 0, 0), (b, 0, 0, 1), (b, 0, 0, 2)]),
(d, 0, 0, 0): (c, 0, 0, 0),
(d, 0, 0, 1): (c, 0, 0, 1),
(d, 0, 1, 0): (c, 0, 1, 0),
(d, 0, 1, 1): (c, 0, 1, 1),
}