Skip to content

Efficient scalable shuffle - P2P shuffle extension #7507

Open
@fjetter

Description

@fjetter

Motivation

Shuffles are an integral part of many distributed data manipulation algorithms. Common DataFrame operations relying on shuffling include sort, merge, set_index, or various groupby operations (e.g. groupby().apply(), groupby(split_out>1)) whereas the most stereotypical array workload is the rechunk. There are many other applications for an efficient shuffle implementation which justifies taking a dedicated approach to solve this issue.

Shuffling is a poor fit for centralized graph-based scheduling, since the graph is all-to-all (naive O(N²), dask O(N logN); in size where N is the number of partitions), yet the core logic of a shuffle is so simple, it benefits little from centralized coordination, while suffering significant overhead from it. With task-based shuffles, the amount of data we can shuffle effectively (before workers run out of memory, or the scheduler crashes or bottlenecks) is severely limited. Allowing workers to autonomously exchange data with their peers and manage disk and memory usage in a more granular way allows us to push that limit significantly higher.

See https://coiled.io/blog/better-shuffling-in-dask-a-proof-of-concept/ for more background.

This issue tracks the current implementation progress and highlights various milestones. We intend to update the top-level description of this issue continuously such that this issue can serve as an always up-to-date overview of the current efforts.

Goals

  • Can reliably shuffle orders-of-magnitude larger datasets (in total size and number of partitions) than the current task-based shuffle
  • Can shuffle larger-than-memory datasets by spilling to disk
  • Constant, predictable memory footprint per worker, which scales linearly with partition size, not total number of partitions
  • Just works, without users needing to tune parameters (buffer sizes, etc.)
  • Graceful restarting when possible, and quick failure when not
  • All state is cleaned up on success, failure, or cancellation
  • Shuffle performance is IO-bound (network, disk)
  • Resilience to worker failures via restart of computation

Roadmap

1 - Foundations and dask.DataFrame ✅

The implementation effort so far focused on creating a stable foundation for the things to come and is deriving from the early prototype. This stage mostly focused on a consistent concurrency model that supports off-band, direct peer to peer communication between workers and integrates well with the existing task based scheduling logic.

This was developed at the example of a DataFrame based shuffle and we consider this now ready to use!

For detailed instructions, known issues and feedback, please see #7509. We encourage all users of dask.DataFrame to try this and report with feedback.

2 - dask.Array rechunking

The new shuffle extension is currently build to handle pandas DataFrames and is using pyarrow behind the scenes. It's architecture is built with generic types in mind and will be suited just as well for array workloads. One of the most popular many-to-many problems is the array rechunking which we will implement next using this extension.

Basic functionality is being set up in #7534

This approach already provides constant time array rechunking but sometimes falls short in terms of walltime performance compared to old style task based shuffling.

3 - Misc

This next stage is not as refined as the intial ones. There are many smaller to medium sized issues that will either expand adoption of the P2P algorithm or make it run faster and smoother. This section will become more refined over time.

Metadata

Metadata

Assignees

No one assigned

    Labels

    No labels
    No labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions