-
-
Notifications
You must be signed in to change notification settings - Fork 728
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Zero-copy array shuffle #8282
Zero-copy array shuffle #8282
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -4,12 +4,15 @@ | |
import pathlib | ||
import shutil | ||
import threading | ||
from collections.abc import Generator | ||
from collections.abc import Callable, Generator, Iterable | ||
from contextlib import contextmanager | ||
from typing import Any, Callable | ||
from typing import Any | ||
|
||
from toolz import concat | ||
|
||
from distributed.shuffle._buffer import ShardsBuffer | ||
from distributed.shuffle._limiter import ResourceLimiter | ||
from distributed.shuffle._pickle import pickle_bytelist | ||
from distributed.utils import Deadline, log_errors | ||
|
||
|
||
|
@@ -135,7 +138,7 @@ def __init__( | |
self._read = read | ||
self._directory_lock = ReadWriteLock() | ||
|
||
async def _process(self, id: str, shards: list[bytes]) -> None: | ||
async def _process(self, id: str, shards: list[Any]) -> None: | ||
"""Write one buffer to file | ||
|
||
This function was built to offload the disk IO, but since then we've | ||
|
@@ -157,11 +160,18 @@ async def _process(self, id: str, shards: list[bytes]) -> None: | |
with self._directory_lock.read(): | ||
if self._closed: | ||
raise RuntimeError("Already closed") | ||
with open( | ||
self.directory / str(id), mode="ab", buffering=100_000_000 | ||
) as f: | ||
for shard in shards: | ||
f.write(shard) | ||
|
||
frames: Iterable[bytes | bytearray | memoryview] | ||
|
||
if not shards or isinstance(shards[0], bytes): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we need to handle There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Happy to change it to There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's give that a shot, if this breaks any tests that would mean something doesn't work as (I) expected. |
||
# Manually serialized dataframes | ||
frames = shards | ||
else: | ||
# Unserialized numpy arrays | ||
frames = concat(pickle_bytelist(shard) for shard in shards) | ||
|
||
with open(self.directory / str(id), mode="ab") as f: | ||
f.writelines(frames) | ||
|
||
def read(self, id: str) -> Any: | ||
"""Read a complete file back into memory""" | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,42 @@ | ||
from __future__ import annotations | ||
|
||
import pickle | ||
from collections.abc import Iterator | ||
from typing import Any | ||
|
||
from distributed.protocol.utils import pack_frames_prelude, unpack_frames | ||
|
||
|
||
def pickle_bytelist(obj: object) -> list[bytes | memoryview]: | ||
"""Variant of :func:`serialize_bytelist`, that doesn't support compression, locally | ||
defined classes, or any of its other fancy features but runs 10x faster for numpy | ||
arrays | ||
|
||
See Also | ||
-------- | ||
serialize_bytelist | ||
unpickle_bytestream | ||
""" | ||
frames: list = [] | ||
pik = pickle.dumps( | ||
obj, protocol=5, buffer_callback=lambda pb: frames.append(pb.raw()) | ||
) | ||
frames.insert(0, pik) | ||
frames.insert(0, pack_frames_prelude(frames)) | ||
return frames | ||
|
||
|
||
def unpickle_bytestream(b: bytes | bytearray | memoryview) -> Iterator[Any]: | ||
"""Unpickle the concatenated output of multiple calls to :func:`pickle_bytelist` | ||
|
||
See Also | ||
-------- | ||
pickle_bytelist | ||
deserialize_bytes | ||
""" | ||
while True: | ||
pik, *buffers, remainder = unpack_frames(b, remainder=True) | ||
yield pickle.loads(pik, buffers=buffers) | ||
if remainder.nbytes == 0: | ||
break | ||
b = remainder |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -96,8 +96,8 @@ | |
|
||
from __future__ import annotations | ||
|
||
import mmap | ||
import os | ||
import pickle | ||
from collections import defaultdict | ||
from collections.abc import Callable, Sequence | ||
from concurrent.futures import ThreadPoolExecutor | ||
|
@@ -123,6 +123,7 @@ | |
handle_unpack_errors, | ||
) | ||
from distributed.shuffle._limiter import ResourceLimiter | ||
from distributed.shuffle._pickle import unpickle_bytestream | ||
from distributed.shuffle._scheduler_plugin import ShuffleSchedulerPlugin | ||
from distributed.shuffle._shuffle import barrier_key, shuffle_barrier | ||
from distributed.shuffle._worker_plugin import ShuffleWorkerPlugin | ||
|
@@ -281,6 +282,9 @@ def convert_chunk(shards: list[list[tuple[NDIndex, np.ndarray]]]) -> np.ndarray: | |
for index, shard in indexed.items(): | ||
rec_cat_arg[tuple(index)] = shard | ||
arrs = rec_cat_arg.tolist() | ||
|
||
# This may block for several seconds, as it physically reads the memory-mapped | ||
# buffers from disk | ||
return concatenate3(arrs) | ||
|
||
|
||
|
@@ -364,72 +368,84 @@ def __init__( | |
self.worker_for = worker_for | ||
self.split_axes = split_axes(old, new) | ||
|
||
async def _receive(self, data: list[tuple[NDIndex, bytes]]) -> None: | ||
async def _receive( | ||
self, | ||
data: list[tuple[NDIndex, list[tuple[NDIndex, tuple[NDIndex, np.ndarray]]]]], | ||
) -> None: | ||
self.raise_if_closed() | ||
|
||
filtered = [] | ||
# Repartition shards and filter out already received ones | ||
shards = defaultdict(list) | ||
for d in data: | ||
id, payload = d | ||
if id in self.received: | ||
id1, payload = d | ||
if id1 in self.received: | ||
continue | ||
filtered.append(payload) | ||
self.received.add(id) | ||
self.received.add(id1) | ||
for id2, shard in payload: | ||
shards[id2].append(shard) | ||
self.total_recvd += sizeof(d) | ||
del data | ||
if not filtered: | ||
if not shards: | ||
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()} | ||
|
||
def _shard_partition( | ||
self, data: np.ndarray, partition_id: NDIndex, **kwargs: Any | ||
) -> dict[str, tuple[NDIndex, bytes]]: | ||
self, data: np.ndarray, partition_id: NDIndex | ||
) -> dict[str, tuple[NDIndex, Any]]: | ||
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) | ||
|
||
shard = data[ndslice] | ||
# Don't wait until all shards have been transferred over the network | ||
# before data can be released | ||
if shard.base is not None: | ||
shard = shard.copy() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. explicit deep-copy mentioned in step 3 of the opening post |
||
|
||
out[self.worker_for[chunk_index]].append( | ||
(chunk_index, (shard_index, data[ndslice])) | ||
(chunk_index, (shard_index, shard)) | ||
) | ||
return {k: (partition_id, pickle.dumps(v)) for k, v in out.items()} | ||
return {k: (partition_id, v) for k, v in out.items()} | ||
|
||
def _get_output_partition( | ||
self, partition_id: NDIndex, key: str, **kwargs: Any | ||
) -> np.ndarray: | ||
# Quickly read metadata from disk. | ||
# This is a bunch of seek()'s interleaved with short reads. | ||
data = self._read_from_disk(partition_id) | ||
return convert_chunk(data) | ||
|
||
def deserialize(self, buffer: bytes) -> Any: | ||
result = pickle.loads(buffer) | ||
return result | ||
|
||
def read(self, path: Path) -> tuple[Any, int]: | ||
shards: list[list[tuple[NDIndex, np.ndarray]]] = [] | ||
with path.open(mode="rb") as f: | ||
size = f.seek(0, os.SEEK_END) | ||
f.seek(0) | ||
while f.tell() < size: | ||
shards.append(pickle.load(f)) | ||
return shards, size | ||
# Copy the memory-mapped buffers from disk into memory. | ||
# This is where we'll spend most time. | ||
with self._disk_buffer.time("read"): | ||
return convert_chunk(data) | ||
|
||
def deserialize(self, buffer: Any) -> Any: | ||
return buffer | ||
|
||
def read(self, path: Path) -> tuple[list[list[tuple[NDIndex, np.ndarray]]], int]: | ||
"""Open a memory-mapped file descriptor to disk, read all metadata, and unpickle | ||
all arrays. This is a fast sequence of short reads interleaved with seeks. | ||
Do not read in memory the actual data; the arrays' buffers will point to the | ||
memory-mapped area. | ||
|
||
The file descriptor will be automatically closed by the kernel when all the | ||
returned arrays are dereferenced, which will happen after the call to | ||
concatenate3. | ||
""" | ||
with path.open(mode="r+b") as fh: | ||
buffer = memoryview(mmap.mmap(fh.fileno(), 0)) | ||
|
||
# The file descriptor has *not* been closed! | ||
shards = list(unpickle_bytestream(buffer)) | ||
return shards, buffer.nbytes | ||
|
||
def _get_assigned_worker(self, id: NDIndex) -> str: | ||
return self.worker_for[id] | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This huge buffering setting was just wasting memory.