Skip to content

Blockwise with concatenate=True broken with LocalCluster + optimize_graph=False  #7449

@JSKenyon

Description

@JSKenyon

What happened:
When using a LocalCluster (and probably the distributed scheduler in general), setting optimize_graph=False causes strange (and incorrect) behaviour in Blockwise with concatentate=True.

What you expected to happen:
Blockwise with concatenate=True should function as normal.

Minimal Complete Verifiable Example:

import dask.array as da
from distributed import LocalCluster, Client


def f(x, y):

    print(y)

    return x


if __name__ == '__main__':

    cluster = LocalCluster(processes=False,
                           n_workers=1,
                           threads_per_worker=1)

    client = Client(cluster)

    x = da.array([0])
    y = da.array([[0]])

    z = da.blockwise(
        f, ("i"),
        x, ("i"),
        y, ("ij"),
        dtype=x.dtype,
        concatenate=True)

    da.compute(z, optimize_graph=False)

Anything else we need to know?:
The example above will print something like:

(<function concatenate_axes at 0x7f153d45c310>, [array([[0]])], [1])

This is not what is expected - we expect an array. Note that the above example will work as normal when:

  • optimize_graph=True
  • A different scheduler is used.

Apologies if this should have been posted on distributed instead.

Environment:

  • Dask version: 2021.3.0
  • Distributed version: 2021.3.0
  • Python version: 3.8.8
  • Operating System: Pop!_OS 18.04 LTS
  • Install method (conda, pip, source): pip

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions