Skip to content

[Dask.order] Memory usage regression for flox xarray reductions #10618

@fjetter

Description

@fjetter

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

image

Tasks are loaded as required and reducers are scheduled promptly. This is nice.

Main (w/ regression)

max_pressure: 8
image

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),
    }

Metadata

Metadata

Assignees

Type

No type

Projects

No projects

Milestone

No milestone

Relationships

None yet

Development

No branches or pull requests

Issue actions