Skip to content

Commit

Permalink
Add some top-level exposition to the p2p rechunking code (#7978)
Browse files Browse the repository at this point in the history
  • Loading branch information
wence- authored Jul 18, 2023
1 parent 9d516da commit 2be7f35
Showing 1 changed file with 96 additions and 0 deletions.
96 changes: 96 additions & 0 deletions distributed/shuffle/_rechunk.py
Original file line number Diff line number Diff line change
@@ -1,3 +1,99 @@
"""
Utilities for rechunking arrays through p2p shuffles
====================================================
Tensors (or n-D arrays) in dask are split up across the workers as
regular n-D "chunks" or bricks. These bricks are stacked up to form
the global array.
A key algorithm for these tensors is to "rechunk" them. That is to
reassemble the same global representation using differently shaped n-D
bricks.
For example, to take an FFT of an n-D array, one uses a sequence of 1D
FFTs along each axis. The implementation in dask (and indeed almost
all distributed array frameworks) requires that 1D
axis along which the FFT is taken is local to a single brick. So to
perform the global FFT we need to arrange that each axis in turn is
local to bricks.
This can be achieved through all-to-all communication between the
workers to exchange sub-pieces of their individual bricks, given a
"rechunking" scheme.
To perform the redistribution, each input brick is cut up into some
number of smaller pieces, each of which contributes to one of the
output bricks. The mapping from input brick to output bricks
decomposes into the Cartesian product of axis-by-axis mappings. To
see this, consider first a 1D example.
Suppose our array is split up into three equally sized bricks::
|----0----|----1----|----2----|
And the requested output chunks are::
|--A--|--B--|----C----|---D---|
So brick 0 contributes to output bricks A and B; brick 1 contributes
to B and C; and brick 2 contributes to C and D.
Now consider a 2D example of the same problem::
+----0----+----1----+----2----+
| | | |
α | | |
| | | |
+---------+---------+---------+
| | | |
β | | |
| | | |
+---------+---------+---------+
| | | |
γ | | |
| | | |
+---------+---------+---------+
Each brick can be described as the ordered pair of row and column
1D bricks, (0, α), (0, β), ..., (2, γ). Since the rechunking does
not also reshape the array, axes do not "interfere" with one another
when determining output bricks::
+--A--+--B--+----C----+---D---+
| | | | |
Σ | | | |
| | | | |
+-----+ ----+---------+-------+
| | | | |
| | | | |
| | | | |
Π | | | |
| | | | |
| | | | |
| | | | |
+-----+-----+---------+-------+
Consider the output (B, Σ) brick. This is contributed to by the
input (0, α) and (1, α) bricks. Determination of the subslices is
just done by slicing the the axes separately and combining them.
The key thing to note here is that we never need to create, and
store, the dense 2D mapping, we can instead construct it on the fly
for each output brick in turn as necessary.
The implementation here uses :func:`split_axes` to construct these
1D rechunkings. The output partitioning in
:meth:`~.ArrayRechunkRun.add_partition` then lazily constructs the
subsection of the Cartesian product it needs to determine the slices
of the current input brick.
This approach relies on the generic p2p buffering machinery to
ensure that there are not too many small messages exchanged, since
no special effort is made to minimise messages between workers when
a worker might have two adjacent input bricks that are sliced into
the same output brick.
"""

from __future__ import annotations

from typing import TYPE_CHECKING, NamedTuple
Expand Down

0 comments on commit 2be7f35

Please sign in to comment.