/
_shuffle.py
544 lines (467 loc) · 16.4 KB
/
_shuffle.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
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
from __future__ import annotations
import logging
import os
from collections import defaultdict
from collections.abc import Callable, Collection, Iterable, Iterator, Sequence
from concurrent.futures import ThreadPoolExecutor
from dataclasses import dataclass
from functools import partial
from pathlib import Path
from typing import TYPE_CHECKING, Any
import toolz
from dask.base import tokenize
from dask.highlevelgraph import HighLevelGraph
from dask.layers import Layer
from dask.typing import Key
from distributed.core import PooledRPCCall
from distributed.shuffle._arrow import (
check_dtype_support,
check_minimal_arrow_version,
convert_shards,
list_of_buffers_to_table,
read_from_disk,
serialize_table,
)
from distributed.shuffle._core import (
NDIndex,
ShuffleId,
ShuffleRun,
ShuffleSpec,
barrier_key,
get_worker_plugin,
handle_transfer_errors,
handle_unpack_errors,
)
from distributed.shuffle._limiter import ResourceLimiter
from distributed.shuffle._scheduler_plugin import ShuffleSchedulerPlugin
from distributed.shuffle._worker_plugin import ShuffleWorkerPlugin
from distributed.sizeof import sizeof
logger = logging.getLogger("distributed.shuffle")
if TYPE_CHECKING:
import pandas as pd
import pyarrow as pa
# TODO import from typing (requires Python >=3.10)
from typing_extensions import TypeAlias
from dask.dataframe import DataFrame
def shuffle_transfer(
input: pd.DataFrame,
id: ShuffleId,
input_partition: int,
npartitions: int,
column: str,
meta: pd.DataFrame,
parts_out: set[int],
) -> int:
with handle_transfer_errors(id):
return get_worker_plugin().add_partition(
input,
input_partition,
spec=DataFrameShuffleSpec(
id=id,
npartitions=npartitions,
column=column,
meta=meta,
parts_out=parts_out,
),
)
def shuffle_unpack(
id: ShuffleId, output_partition: int, barrier_run_id: int
) -> pd.DataFrame:
with handle_unpack_errors(id):
return get_worker_plugin().get_output_partition(
id, barrier_run_id, output_partition
)
def shuffle_barrier(id: ShuffleId, run_ids: list[int]) -> int:
try:
return get_worker_plugin().barrier(id, run_ids)
except Exception as e:
raise RuntimeError(f"shuffle_barrier failed during shuffle {id}") from e
def rearrange_by_column_p2p(
df: DataFrame,
column: str,
npartitions: int | None = None,
) -> DataFrame:
import pandas as pd
from dask.dataframe.core import new_dd_object
meta = df._meta
if not pd.api.types.is_integer_dtype(meta[column].dtype):
raise TypeError(
f"Expected meta {column=} to be an integer column, is {meta[column].dtype}."
)
check_dtype_support(meta)
npartitions = npartitions or df.npartitions
token = tokenize(df, column, npartitions)
if any(not isinstance(c, str) for c in meta.columns):
unsupported = {c: type(c) for c in meta.columns if not isinstance(c, str)}
raise TypeError(
f"p2p requires all column names to be str, found: {unsupported}",
)
name = f"shuffle_p2p-{token}"
layer = P2PShuffleLayer(
name,
column,
npartitions,
npartitions_input=df.npartitions,
name_input=df._name,
meta_input=meta,
)
return new_dd_object(
HighLevelGraph.from_collections(name, layer, [df]),
name,
meta,
[None] * (npartitions + 1),
)
_T_LowLevelGraph: TypeAlias = dict[Key, tuple]
class P2PShuffleLayer(Layer):
def __init__(
self,
name: str,
column: str,
npartitions: int,
npartitions_input: int,
name_input: str,
meta_input: pd.DataFrame,
parts_out: Iterable | None = None,
annotations: dict | None = None,
):
check_minimal_arrow_version()
self.name = name
self.column = column
self.npartitions = npartitions
self.name_input = name_input
self.meta_input = meta_input
if parts_out:
self.parts_out = set(parts_out)
else:
self.parts_out = set(range(self.npartitions))
self.npartitions_input = npartitions_input
annotations = annotations or {}
annotations.update({"shuffle": lambda key: key[1]})
super().__init__(annotations=annotations)
def __repr__(self) -> str:
return (
f"{type(self).__name__}<name='{self.name}', npartitions={self.npartitions}>"
)
def get_output_keys(self) -> set[Key]:
return {(self.name, part) for part in self.parts_out}
def is_materialized(self) -> bool:
return hasattr(self, "_cached_dict")
@property
def _dict(self) -> _T_LowLevelGraph:
"""Materialize full dict representation"""
self._cached_dict: _T_LowLevelGraph
dsk: _T_LowLevelGraph
if hasattr(self, "_cached_dict"):
return self._cached_dict
else:
dsk = self._construct_graph()
self._cached_dict = dsk
return self._cached_dict
def __getitem__(self, key: Key) -> tuple:
return self._dict[key]
def __iter__(self) -> Iterator[Key]:
return iter(self._dict)
def __len__(self) -> int:
return len(self._dict)
def _cull(self, parts_out: Iterable[int]) -> P2PShuffleLayer:
return P2PShuffleLayer(
self.name,
self.column,
self.npartitions,
self.npartitions_input,
self.name_input,
self.meta_input,
parts_out=parts_out,
)
def _keys_to_parts(self, keys: Iterable[Key]) -> set[int]:
"""Simple utility to convert keys to partition indices."""
parts = set()
for key in keys:
if isinstance(key, tuple) and len(key) == 2:
name, part = key
if name == self.name:
assert isinstance(part, int)
parts.add(part)
return parts
def cull(
self, keys: set[Key], all_keys: Collection[Key]
) -> tuple[P2PShuffleLayer, dict]:
"""Cull a P2PShuffleLayer HighLevelGraph layer.
The underlying graph will only include the necessary
tasks to produce the keys (indices) included in `parts_out`.
Therefore, "culling" the layer only requires us to reset this
parameter.
"""
parts_out = self._keys_to_parts(keys)
# Protect against mutations later on with frozenset
input_parts = frozenset(
{(self.name_input, i) for i in range(self.npartitions_input)}
)
culled_deps = {(self.name, part): input_parts for part in parts_out}
if parts_out != set(self.parts_out):
culled_layer = self._cull(parts_out)
return culled_layer, culled_deps
else:
return self, culled_deps
def _construct_graph(self) -> _T_LowLevelGraph:
token = tokenize(self.name_input, self.column, self.npartitions, self.parts_out)
dsk: _T_LowLevelGraph = {}
_barrier_key = barrier_key(ShuffleId(token))
name = "shuffle-transfer-" + token
transfer_keys = list()
for i in range(self.npartitions_input):
transfer_keys.append((name, i))
dsk[(name, i)] = (
shuffle_transfer,
(self.name_input, i),
token,
i,
self.npartitions,
self.column,
self.meta_input,
self.parts_out,
)
dsk[_barrier_key] = (shuffle_barrier, token, transfer_keys)
name = self.name
for part_out in self.parts_out:
dsk[(name, part_out)] = (
shuffle_unpack,
token,
part_out,
_barrier_key,
)
return dsk
def split_by_worker(
df: pd.DataFrame,
column: str,
meta: pd.DataFrame,
worker_for: pd.Series,
) -> dict[Any, pa.Table]:
"""
Split data into many arrow batches, partitioned by destination worker
"""
import numpy as np
from dask.dataframe.dispatch import to_pyarrow_table_dispatch
df = df.astype(meta.dtypes, copy=False)
# (cudf support) Avoid pd.Series
constructor = df._constructor_sliced
assert isinstance(constructor, type)
worker_for = constructor(worker_for)
df = df.merge(
right=worker_for.cat.codes.rename("_worker"),
left_on=column,
right_index=True,
how="inner",
)
nrows = len(df)
if not nrows:
return {}
# assert len(df) == nrows # Not true if some outputs aren't wanted
# FIXME: If we do not preserve the index something is corrupting the
# bytestream such that it cannot be deserialized anymore
t = to_pyarrow_table_dispatch(df, preserve_index=True)
t = t.sort_by("_worker")
codes = np.asarray(t["_worker"])
t = t.drop(["_worker"])
del df
splits = np.where(codes[1:] != codes[:-1])[0] + 1
splits = np.concatenate([[0], splits])
shards = [
t.slice(offset=a, length=b - a) for a, b in toolz.sliding_window(2, splits)
]
shards.append(t.slice(offset=splits[-1], length=None))
unique_codes = codes[splits]
out = {
# FIXME https://github.com/pandas-dev/pandas-stubs/issues/43
worker_for.cat.categories[code]: shard
for code, shard in zip(unique_codes, shards)
}
assert sum(map(len, out.values())) == nrows
return out
def split_by_partition(t: pa.Table, column: str) -> dict[int, pa.Table]:
"""
Split data into many arrow batches, partitioned by final partition
"""
import numpy as np
partitions = t.select([column]).to_pandas()[column].unique()
partitions.sort()
t = t.sort_by(column)
partition = np.asarray(t[column])
splits = np.where(partition[1:] != partition[:-1])[0] + 1
splits = np.concatenate([[0], splits])
shards = [
t.slice(offset=a, length=b - a) for a, b in toolz.sliding_window(2, splits)
]
shards.append(t.slice(offset=splits[-1], length=None))
assert len(t) == sum(map(len, shards))
assert len(partitions) == len(shards)
return dict(zip(partitions, shards))
class DataFrameShuffleRun(ShuffleRun[int, "pd.DataFrame"]):
"""State for a single active shuffle 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 shuffle
with other run instances using `rpc`.
The user of this needs to guarantee that only `DataFrameShuffleRun`s of the
same unique `ShuffleID` and `run_id` interact.
Parameters
----------
worker_for:
A mapping partition_id -> worker_address.
column:
The data column we split the input partition by.
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.
"""
column: str
meta: pd.DataFrame
partitions_of: dict[str, list[int]]
worker_for: pd.Series
def __init__(
self,
worker_for: dict[int, str],
column: str,
meta: pd.DataFrame,
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,
):
import pandas as pd
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.column = column
self.meta = meta
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 = pd.Series(worker_for, name="_workers").astype("category")
async def receive(self, data: list[tuple[int, bytes]]) -> None:
await self._receive(data)
async def _receive(self, data: list[tuple[int, bytes]]) -> None:
self.raise_if_closed()
filtered = []
for d in data:
if d[0] not in self.received:
filtered.append(d[1])
self.received.add(d[0])
self.total_recvd += sizeof(d)
del data
if not filtered:
return
try:
groups = await self.offload(self._repartition_buffers, filtered)
del filtered
await self._write_to_disk(groups)
except Exception as e:
self._exception = e
raise
def _repartition_buffers(self, data: list[bytes]) -> dict[NDIndex, bytes]:
table = list_of_buffers_to_table(data)
groups = split_by_partition(table, self.column)
assert len(table) == sum(map(len, groups.values()))
del data
return {(k,): serialize_table(v) for k, v in groups.items()}
async def _add_partition(
self,
data: pd.DataFrame,
partition_id: int,
**kwargs: Any,
) -> int:
def _() -> dict[str, tuple[int, bytes]]:
out = split_by_worker(
data,
self.column,
self.meta,
self.worker_for,
)
out = {k: (partition_id, serialize_table(t)) for k, t in out.items()}
return out
out = await self.offload(_)
await self._write_to_comm(out)
return self.run_id
async def _get_output_partition(
self,
partition_id: int,
key: str,
**kwargs: Any,
) -> pd.DataFrame:
try:
def _(partition_id: int, meta: pd.DataFrame) -> pd.DataFrame:
data = self._read_from_disk((partition_id,))
return convert_shards(data, meta)
out = await self.offload(_, partition_id, self.meta)
except KeyError:
out = self.meta.copy()
return out
def _get_assigned_worker(self, id: int) -> str:
return self.worker_for[id]
def read(self, path: Path) -> tuple[Any, int]:
return read_from_disk(path, self.meta)
@dataclass(frozen=True)
class DataFrameShuffleSpec(ShuffleSpec[int]):
npartitions: int
column: str
meta: pd.DataFrame
parts_out: set[int]
def _pin_output_workers(self, plugin: ShuffleSchedulerPlugin) -> dict[int, str]:
pick_worker = partial(_get_worker_for_range_sharding, self.npartitions)
return plugin._pin_output_workers(self.id, self.parts_out, pick_worker)
def create_run_on_worker(
self, run_id: int, worker_for: dict[int, str], plugin: ShuffleWorkerPlugin
) -> ShuffleRun:
return DataFrameShuffleRun(
column=self.column,
meta=self.meta,
worker_for=worker_for,
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_range_sharding(
npartitions: int, output_partition: int, workers: Sequence[str]
) -> str:
"""Get address of target worker for this output partition using range sharding"""
i = len(workers) * output_partition // npartitions
return workers[i]