Skip to content

da.blockwise(concatenate=True) surprising behavior with contraction. #6340

@Carreau

Description

@Carreau

When using concatenate=True it seem like blockwise replicate and concatenate arrays with dimension 1 (that would have broadcasted), as many time as there are chunks in the other arrays – which make sens with concatenate=True, but not concatenate=False

import dask.array as da
p,k,N = 3,2,10
r = da.blockwise(
    lambda a, b: print("a.shape =", a.shape, "and b.shape =", b.shape, '[expected (',p,',1)]'),
    "x",
    da.zeros((p, k*N), chunks=(p, k)),
    "xz",
    da.zeros((p, 1), chunks=-1),
    "xz",
    concatenate=True,
    dtype=float,
).compute()

prints

[snip]
a.shape = (3, 20) and b.shape = (3, 10) [expected ( 3 ,1)]

Anything else we need to know?:

I believe this is an interaction between trying to mimic broadcasting across chunks and concatenate concatenating late in the graph.

  1. is one of the array has a dimension of one, and need to be contracted, it will be replicated to be able to be broadcasted to each chunks independently across the dimension of broadcast, so that the contracting function can "just" zip across its items and numpy will take care of the broadcasting.

  2. if concat=True, somethings (concatenate_axes ) just take all the elements in the iterators and concatenate them – though this is now incorrect as the arrays was replicated along the broadcast axis.

Environment:

  • Dask version: 2.18.1+13
  • Python version: 3.8, anaconda
  • Operating System: Macos
  • Install method (conda, pip, source): source.

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions