/
_rechunk.py
493 lines (407 loc) · 16.2 KB
/
_rechunk.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
"""
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
import os
import pickle
from collections import defaultdict
from collections.abc import Callable, Sequence
from concurrent.futures import ThreadPoolExecutor
from dataclasses import dataclass
from io import BytesIO
from itertools import product
from typing import TYPE_CHECKING, Any, NamedTuple
import dask
from dask.base import tokenize
from dask.highlevelgraph import HighLevelGraph, MaterializedLayer
from distributed.core import PooledRPCCall
from distributed.exceptions import Reschedule
from distributed.shuffle._core import (
NDIndex,
ShuffleId,
ShuffleRun,
ShuffleSpec,
get_worker_plugin,
)
from distributed.shuffle._limiter import ResourceLimiter
from distributed.shuffle._scheduler_plugin import ShuffleSchedulerPlugin
from distributed.shuffle._shuffle import barrier_key, shuffle_barrier
from distributed.shuffle._worker_plugin import ShuffleWorkerPlugin
from distributed.sizeof import sizeof
if TYPE_CHECKING:
import numpy as np
from typing_extensions import TypeAlias
import dask.array as da
ChunkedAxis: TypeAlias = tuple[float, ...] # chunks must either be an int or NaN
ChunkedAxes: TypeAlias = tuple[ChunkedAxis, ...]
NDSlice: TypeAlias = tuple[slice, ...]
def rechunk_transfer(
input: np.ndarray,
id: ShuffleId,
input_chunk: NDIndex,
new: ChunkedAxes,
old: ChunkedAxes,
) -> int:
try:
return get_worker_plugin().add_partition(
input,
partition_id=input_chunk,
spec=ArrayRechunkSpec(id=id, new=new, old=old),
)
except Exception as e:
raise RuntimeError(f"rechunk_transfer failed during shuffle {id}") from e
def rechunk_unpack(
id: ShuffleId, output_chunk: NDIndex, barrier_run_id: int
) -> np.ndarray:
try:
return get_worker_plugin().get_output_partition(
id, barrier_run_id, output_chunk
)
except Reschedule as e:
raise e
except Exception as e:
raise RuntimeError(f"rechunk_unpack failed during shuffle {id}") from e
def rechunk_p2p(x: da.Array, chunks: ChunkedAxes) -> da.Array:
import numpy as np
import dask.array as da
if x.size == 0:
# Special case for empty array, as the algorithm below does not behave correctly
return da.empty(x.shape, chunks=chunks, dtype=x.dtype)
dsk: dict = {}
token = tokenize(x, chunks)
_barrier_key = barrier_key(ShuffleId(token))
name = f"rechunk-transfer-{token}"
transfer_keys = []
for index in np.ndindex(tuple(len(dim) for dim in x.chunks)):
transfer_keys.append((name,) + index)
dsk[(name,) + index] = (
rechunk_transfer,
(x.name,) + index,
token,
index,
chunks,
x.chunks,
)
dsk[_barrier_key] = (shuffle_barrier, token, transfer_keys)
name = f"rechunk-p2p-{token}"
for index in np.ndindex(tuple(len(dim) for dim in chunks)):
dsk[(name,) + index] = (rechunk_unpack, token, index, _barrier_key)
with dask.annotate(shuffle=lambda key: key[1:]):
layer = MaterializedLayer(dsk)
graph = HighLevelGraph.from_collections(name, layer, dependencies=[x])
return da.Array(graph, name, chunks, meta=x)
class Split(NamedTuple):
"""Slice of a chunk that is concatenated with other splits to create a new chunk
Splits define how to slice an input chunk on a single axis into small pieces
that can be concatenated together with splits from other input chunks to create
output chunks of a rechunk operation.
"""
#: Index of the new output chunk to which this split belongs.
chunk_index: int
#: Index of the split within the list of splits that are concatenated
#: to create the new chunk.
split_index: int
#: Slice of the input chunk.
slice: slice
SplitChunk: TypeAlias = list[Split]
SplitAxis: TypeAlias = list[SplitChunk]
SplitAxes: TypeAlias = list[SplitAxis]
def split_axes(old: ChunkedAxes, new: ChunkedAxes) -> SplitAxes:
"""Calculate how to split the old chunks on each axis to create the new chunks
Parameters
----------
old : ChunkedAxes
Chunks along each axis of the old array
new : ChunkedAxes
Chunks along each axis of the new array
Returns
-------
SplitAxes
Splits along each axis that determine how to slice the input chunks to create
the new chunks by concatenating the resulting shards.
"""
from dask.array.rechunk import old_to_new
_old_to_new = old_to_new(old, new)
axes = []
for axis_index, new_axis in enumerate(_old_to_new):
old_axis: SplitAxis = [[] for _ in old[axis_index]]
for new_chunk_index, new_chunk in enumerate(new_axis):
for split_index, (old_chunk_index, slice) in enumerate(new_chunk):
old_axis[old_chunk_index].append(
Split(new_chunk_index, split_index, slice)
)
for old_chunk in old_axis:
old_chunk.sort(key=lambda split: split.slice.start)
axes.append(old_axis)
return axes
def convert_chunk(data: bytes) -> np.ndarray:
import numpy as np
from dask.array.core import concatenate3
file = BytesIO(data)
shards: dict[NDIndex, np.ndarray] = {}
while file.tell() < len(data):
for index, shard in pickle.load(file):
shards[index] = shard
subshape = [max(dim) + 1 for dim in zip(*shards.keys())]
assert len(shards) == np.prod(subshape)
rec_cat_arg = np.empty(subshape, dtype="O")
for index, shard in shards.items():
rec_cat_arg[tuple(index)] = shard
del data
del file
arrs = rec_cat_arg.tolist()
return concatenate3(arrs)
class ArrayRechunkRun(ShuffleRun[NDIndex, "np.ndarray"]):
"""State for a single active rechunk execution
This object is responsible for splitting, sending, receiving and combining
data shards.
It is entirely agnostic to the distributed system and can perform a rechunk
with other run instances using `rpc``.
The user of this needs to guarantee that only `ArrayRechunkRun`s of the same unique
`ShuffleID` and `run_id` interact.
Parameters
----------
worker_for:
A mapping partition_id -> worker_address.
old:
Existing chunking of the array per dimension.
new:
Desired chunking of the array per dimension.
id:
A unique `ShuffleID` this belongs to.
run_id:
A unique identifier of the specific execution of the shuffle this belongs to.
local_address:
The local address this Shuffle can be contacted by using `rpc`.
directory:
The scratch directory to buffer data in.
executor:
Thread pool to use for offloading compute.
rpc:
A callable returning a PooledRPCCall to contact other Shuffle instances.
Typically a ConnectionPool.
scheduler:
A PooledRPCCall to to contact the scheduler.
memory_limiter_disk:
memory_limiter_comm:
A ``ResourceLimiter`` limiting the total amount of memory used in either
buffer.
"""
def __init__(
self,
worker_for: dict[NDIndex, str],
old: ChunkedAxes,
new: ChunkedAxes,
id: ShuffleId,
run_id: int,
local_address: str,
directory: str,
executor: ThreadPoolExecutor,
rpc: Callable[[str], PooledRPCCall],
scheduler: PooledRPCCall,
memory_limiter_disk: ResourceLimiter,
memory_limiter_comms: ResourceLimiter,
):
super().__init__(
id=id,
run_id=run_id,
local_address=local_address,
directory=directory,
executor=executor,
rpc=rpc,
scheduler=scheduler,
memory_limiter_comms=memory_limiter_comms,
memory_limiter_disk=memory_limiter_disk,
)
self.old = old
self.new = new
partitions_of = defaultdict(list)
for part, addr in worker_for.items():
partitions_of[addr].append(part)
self.partitions_of = dict(partitions_of)
self.worker_for = worker_for
self.split_axes = split_axes(old, new)
async def _receive(self, data: list[tuple[NDIndex, bytes]]) -> None:
self.raise_if_closed()
filtered = []
for d in data:
id, payload = d
if id in self.received:
continue
filtered.append(payload)
self.received.add(id)
self.total_recvd += sizeof(d)
del data
if not filtered:
return
try:
shards = await self.offload(self._repartition_shards, filtered)
del filtered
await self._write_to_disk(shards)
except Exception as e:
self._exception = e
raise
def _repartition_shards(self, data: list[bytes]) -> dict[NDIndex, bytes]:
repartitioned: defaultdict[
NDIndex, list[tuple[NDIndex, np.ndarray]]
] = defaultdict(list)
for buffer in data:
for id, shard in pickle.loads(buffer):
repartitioned[id].append(shard)
return {k: pickle.dumps(v) for k, v in repartitioned.items()}
async def add_partition(
self, data: np.ndarray, partition_id: NDIndex, **kwargs: Any
) -> int:
self.raise_if_closed()
if self.transferred:
raise RuntimeError(f"Cannot add more partitions to {self}")
def _() -> dict[str, tuple[NDIndex, bytes]]:
"""Return a mapping of worker addresses to a tuple of input partition
IDs and shard data.
TODO: Overhaul!
As shard data, we serialize the payload together with the sub-index of the
slice within the new chunk. To assemble the new chunk from its shards, it
needs the sub-index to know where each shard belongs within the chunk.
Adding the sub-index into the serialized payload on the sender allows us to
write the serialized payload directly to disk on the receiver.
"""
out: dict[
str, list[tuple[NDIndex, tuple[NDIndex, np.ndarray]]]
] = defaultdict(list)
from itertools import product
ndsplits = product(
*(axis[i] for axis, i in zip(self.split_axes, partition_id))
)
for ndsplit in ndsplits:
chunk_index, shard_index, ndslice = zip(*ndsplit)
out[self.worker_for[chunk_index]].append(
(chunk_index, (shard_index, data[ndslice]))
)
return {k: (partition_id, pickle.dumps(v)) for k, v in out.items()}
out = await self.offload(_)
await self._write_to_comm(out)
return self.run_id
async def get_output_partition(
self, partition_id: NDIndex, key: str, **kwargs: Any
) -> np.ndarray:
self.raise_if_closed()
if not self.transferred:
raise RuntimeError("`get_output_partition` called before barrier task")
await self._ensure_output_worker(partition_id, key)
await self.flush_receive()
data = self._read_from_disk(partition_id)
return await self.offload(convert_chunk, data)
def _get_assigned_worker(self, id: NDIndex) -> str:
return self.worker_for[id]
@dataclass(frozen=True)
class ArrayRechunkSpec(ShuffleSpec[NDIndex]):
new: ChunkedAxes
old: ChunkedAxes
def _pin_output_workers(self, plugin: ShuffleSchedulerPlugin) -> dict[NDIndex, str]:
parts_out = product(*(range(len(c)) for c in self.new))
return plugin._pin_output_workers(
self.id, parts_out, _get_worker_for_hash_sharding
)
def create_run_on_worker(
self,
run_id: int,
worker_for: dict[NDIndex, str],
plugin: ShuffleWorkerPlugin,
) -> ShuffleRun:
return ArrayRechunkRun(
worker_for=worker_for,
old=self.old,
new=self.new,
id=self.id,
run_id=run_id,
directory=os.path.join(
plugin.worker.local_directory,
f"shuffle-{self.id}-{run_id}",
),
executor=plugin._executor,
local_address=plugin.worker.address,
rpc=plugin.worker.rpc,
scheduler=plugin.worker.scheduler,
memory_limiter_disk=plugin.memory_limiter_disk,
memory_limiter_comms=plugin.memory_limiter_comms,
)
def _get_worker_for_hash_sharding(
output_partition: NDIndex, workers: Sequence[str]
) -> str:
"""Get address of target worker for this output partition using hash sharding"""
i = hash(output_partition) % len(workers)
return workers[i]