-
-
Notifications
You must be signed in to change notification settings - Fork 748
[RFC]: Allow in-memory buffering of p2p shuffle results #7618
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
Draft
wence-
wants to merge
5
commits into
dask:main
Choose a base branch
from
wence-:wence/fea/p2p-buffer-config
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Draft
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
eaf8aea
p2p: Add a no-op limiter to simplify control flow
wence- 0b97fcf
Introduce abstract FileShardsBuffer and MemoryShardsBuffer
wence- 7e940c0
Introduce a Protocol for limiters
wence- 9cd3c14
Add option to buffer shuffle outputs in memory
wence- f4f05fb
Merge branch 'main' into pr/wence-/7618
hendrikmakait File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
|
|
@@ -3,53 +3,68 @@ | |||||
| import contextlib | ||||||
| import pathlib | ||||||
| import shutil | ||||||
| from collections import defaultdict | ||||||
| from io import BytesIO | ||||||
| from types import TracebackType | ||||||
| from typing import BinaryIO | ||||||
|
|
||||||
| from distributed.shuffle._buffer import ShardsBuffer | ||||||
| from distributed.shuffle._limiter import ResourceLimiter | ||||||
| from distributed.utils import log_errors | ||||||
|
|
||||||
|
|
||||||
| class DiskShardsBuffer(ShardsBuffer): | ||||||
| """Accept, buffer, and write many small objects to many files | ||||||
|
|
||||||
| This takes in lots of small objects, writes them to a local directory, and | ||||||
| then reads them back when all writes are complete. It buffers these | ||||||
| objects in memory so that it can optimize disk access for larger writes. | ||||||
|
|
||||||
| **State** | ||||||
|
|
||||||
| - shards: dict[str, list[bytes]] | ||||||
|
|
||||||
| This is our in-memory buffer of data waiting to be written to files. | ||||||
|
|
||||||
| - sizes: dict[str, int] | ||||||
|
|
||||||
| The size of each list of shards. We find the largest and write data from that buffer | ||||||
| class FileShardsBuffer(ShardsBuffer): | ||||||
| """An abstract buffering object backed by a "file" | ||||||
|
|
||||||
| Parameters | ||||||
| ---------- | ||||||
| directory : str or pathlib.Path | ||||||
| Where to write and read data. Ideally points to fast disk. | ||||||
| memory_limiter : ResourceLimiter, optional | ||||||
| Limiter for in-memory buffering (at most this much data) | ||||||
| before writes to disk occur. If the incoming data that has yet | ||||||
| to be processed exceeds this limit, then the buffer will block | ||||||
| until below the threshold. See :meth:`.write` for the | ||||||
| implementation of this scheme. | ||||||
| Resource limiter. | ||||||
|
|
||||||
| Notes | ||||||
| ----- | ||||||
| Currently, a concurrency limit of one is hard-coded. | ||||||
| """ | ||||||
|
|
||||||
| def __init__( | ||||||
| self, | ||||||
| directory: str | pathlib.Path, | ||||||
| memory_limiter: ResourceLimiter | None = None, | ||||||
| ): | ||||||
| def __init__(self, memory_limiter: ResourceLimiter | None = None) -> None: | ||||||
| super().__init__( | ||||||
| memory_limiter=memory_limiter, | ||||||
| # Disk is not able to run concurrently atm | ||||||
| # FileShardsBuffer not able to run concurrently | ||||||
| concurrency_limit=1, | ||||||
| ) | ||||||
| self.directory = pathlib.Path(directory) | ||||||
| self.directory.mkdir(exist_ok=True) | ||||||
|
|
||||||
| def writer(self, id: int | str) -> BinaryIO: | ||||||
| """Return a file-like object for writing in append-mode. | ||||||
|
|
||||||
| Parameters | ||||||
| ---------- | ||||||
| id | ||||||
| The shard id (will normalised to a string) | ||||||
|
|
||||||
| Returns | ||||||
| ------- | ||||||
| An object implementing the BinaryIO interface. | ||||||
| """ | ||||||
| raise NotImplementedError("Abstract class can't provide this") | ||||||
|
|
||||||
| def reader(self, id: int | str) -> BinaryIO: | ||||||
| """Return a file-like object for reading from byte-0. | ||||||
|
|
||||||
| Parameters | ||||||
| ---------- | ||||||
| id | ||||||
| The shard id (will be normalised to a string) | ||||||
|
|
||||||
| Returns | ||||||
| ------- | ||||||
| An object implementing the BinaryIO interface. | ||||||
|
|
||||||
| Raises | ||||||
| ------ | ||||||
| FileNotFoundError | ||||||
| If no shard with requested id exists. | ||||||
| """ | ||||||
| raise NotImplementedError("Abstract class can't provide this") | ||||||
|
|
||||||
| async def _process(self, id: str, shards: list[bytes]) -> None: | ||||||
| """Write one buffer to file | ||||||
|
|
@@ -68,9 +83,7 @@ async def _process(self, id: str, shards: list[bytes]) -> None: | |||||
| with log_errors(): | ||||||
| # Consider boosting total_size a bit here to account for duplication | ||||||
| with self.time("write"): | ||||||
| with open( | ||||||
| self.directory / str(id), mode="ab", buffering=100_000_000 | ||||||
| ) as f: | ||||||
| with self.writer(id) as f: | ||||||
| for shard in shards: | ||||||
| f.write(shard) | ||||||
|
|
||||||
|
|
@@ -82,9 +95,7 @@ def read(self, id: int | str) -> bytes: | |||||
|
|
||||||
| try: | ||||||
| with self.time("read"): | ||||||
| with open( | ||||||
| self.directory / str(id), mode="rb", buffering=100_000_000 | ||||||
| ) as f: | ||||||
| with self.reader(id) as f: | ||||||
| data = f.read() | ||||||
| size = f.tell() | ||||||
| except FileNotFoundError: | ||||||
|
|
@@ -96,6 +107,94 @@ def read(self, id: int | str) -> bytes: | |||||
| else: | ||||||
| raise KeyError(id) | ||||||
|
|
||||||
|
|
||||||
| class _PersistentBytesIO(BytesIO): | ||||||
| """A BytesIO object that does not close itself when used in a with block.""" | ||||||
|
|
||||||
| def __enter__(self) -> _PersistentBytesIO: | ||||||
| return self | ||||||
|
|
||||||
| def __exit__( | ||||||
| self, | ||||||
| exc_type: type[BaseException] | None, | ||||||
| exc_val: BaseException | None, | ||||||
| exc_tb: TracebackType | None, | ||||||
| ) -> None: | ||||||
| pass | ||||||
|
|
||||||
|
|
||||||
| class MemoryShardsBuffer(FileShardsBuffer): | ||||||
| """Accept and buffer many small objects into memory. | ||||||
|
|
||||||
| This implements in-memory "file" buffering with no resource limit | ||||||
| with the same interface as :class:`DiskShardsBuffer`. | ||||||
|
|
||||||
| """ | ||||||
|
|
||||||
| def __init__(self) -> None: | ||||||
| super().__init__(memory_limiter=None) | ||||||
| self._memory_buffers: defaultdict[str, _PersistentBytesIO] = defaultdict( | ||||||
| _PersistentBytesIO | ||||||
| ) | ||||||
|
|
||||||
| def writer(self, id: int | str) -> BinaryIO: | ||||||
| buf = self._memory_buffers[str(id)] | ||||||
| buf.seek(buf.tell()) | ||||||
| return buf | ||||||
|
|
||||||
| def reader(self, id: int | str) -> BinaryIO: | ||||||
| key = str(id) | ||||||
| if key not in self._memory_buffers: | ||||||
| raise FileNotFoundError(f"Shard with {id=} is unknown") | ||||||
| buf = self._memory_buffers[str(id)] | ||||||
|
Member
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.
Suggested change
This will remove the |
||||||
| buf.seek(0) | ||||||
| return buf | ||||||
|
|
||||||
|
|
||||||
| class DiskShardsBuffer(FileShardsBuffer): | ||||||
| """Accept, buffer, and write many small objects to many files | ||||||
|
|
||||||
| This takes in lots of small objects, writes them to a local directory, and | ||||||
| then reads them back when all writes are complete. It buffers these | ||||||
| objects in memory so that it can optimize disk access for larger writes. | ||||||
|
|
||||||
| **State** | ||||||
|
|
||||||
| - shards: dict[str, list[bytes]] | ||||||
|
|
||||||
| This is our in-memory buffer of data waiting to be written to files. | ||||||
|
|
||||||
| - sizes: dict[str, int] | ||||||
|
|
||||||
| The size of each list of shards. We find the largest and write data from that buffer | ||||||
|
|
||||||
| Parameters | ||||||
| ---------- | ||||||
| directory : str or pathlib.Path | ||||||
| Where to write and read data. Ideally points to fast disk. | ||||||
| memory_limiter : ResourceLimiter, optional | ||||||
| Limiter for in-memory buffering (at most this much data) | ||||||
| before writes to disk occur. If the incoming data that has yet | ||||||
| to be processed exceeds this limit, then the buffer will block | ||||||
| until below the threshold. See :meth:`.write` for the | ||||||
| implementation of this scheme. | ||||||
| """ | ||||||
|
|
||||||
| def __init__( | ||||||
| self, | ||||||
| directory: str | pathlib.Path, | ||||||
| memory_limiter: ResourceLimiter | None = None, | ||||||
| ): | ||||||
| super().__init__(memory_limiter=memory_limiter) | ||||||
| self.directory = pathlib.Path(directory) | ||||||
| self.directory.mkdir(exist_ok=True) | ||||||
|
|
||||||
| def writer(self, id: int | str) -> BinaryIO: | ||||||
| return open(self.directory / str(id), mode="ab", buffering=100_000_000) | ||||||
|
|
||||||
| def reader(self, id: int | str) -> BinaryIO: | ||||||
| return open(self.directory / str(id), mode="rb", buffering=100_000_000) | ||||||
|
|
||||||
| async def close(self) -> None: | ||||||
| await super().close() | ||||||
| with contextlib.suppress(FileNotFoundError): | ||||||
|
|
||||||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Instead of configuring this at shuffle execution time on the worker, we should instead configure this at graph creation time. This would allow us to specify this on a per-shuffle basis.