<?xml version='1.0' encoding='UTF-8'?>
<feed xmlns="http://www.w3.org/2005/Atom" xml:lang="en">
  <id>https://blog.dask.org</id>
  <title>Dask Working Notes - Posts tagged p2p</title>
  <updated>2026-03-05T15:05:26.678364+00:00</updated>
  <link href="https://blog.dask.org"/>
  <link href="https://blog.dask.org/blog/tag/p2p/atom.xml" rel="self"/>
  <generator uri="https://ablog.readthedocs.io/" version="0.11.12">ABlog</generator>
  <entry>
    <id>https://blog.dask.org/2023/03/15/shuffling-large-data-at-constant-memory/</id>
    <title>Shuffling large data at constant memory in Dask</title>
    <updated>2023-03-15T00:00:00+00:00</updated>
    <author>
      <name>Hendrik Makait</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This work was engineered and supported by &lt;a class="reference external" href="https://coiled.io?utm_source=dask-blog&amp;amp;amp;utm_medium=shuffling-large-data-at-constant-memory"&gt;Coiled&lt;/a&gt;. In particular, thanks to &lt;a class="reference external" href="https://github.com/fjetter"&gt;Florian Jetter&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/gjoseph92"&gt;Gabe Joseph&lt;/a&gt;, &lt;a class="reference external" href="https://github.com/hendrikmakait"&gt;Hendrik Makait&lt;/a&gt;, and &lt;a class="reference external" href="https://matthewrocklin.com/?utm_source=dask-blog&amp;amp;amp;utm_medium=shuffling-large-data-at-constant-memory"&gt;Matt Rocklin&lt;/a&gt;. Original version of this post appears on &lt;a class="reference external" href="https://blog.coiled.io/blog/shuffling-large-data-at-constant-memory.html?utm_source=dask-blog&amp;amp;amp;utm_medium=shuffling-large-data-at-constant-memory"&gt;blog.coiled.io&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;With release &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2023.2.1&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; introduces a new shuffling method called P2P, making sorts, merges, and joins faster and using constant memory.
Benchmarks show impressive improvements:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="P2P shuffling uses constant memory while task-based shuffling scales linearly." src="/images/p2p_benchmark_memory_usage.png" style="width: 100%;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;P2P shuffling (blue) uses constant memory while task-based shuffling (orange) scales linearly.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;This article describes the problem, the new solution, and the impact on performance.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 24)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="what-is-shuffling"&gt;

&lt;p&gt;Shuffling is a key primitive in data processing systems.
It is used whenever we move a dataset around in an all-to-all fashion, such as occurs in sorting, dataframe joins, or array rechunking.
Shuffling is a challenging computation to run efficiently, with lots of small tasks sharding the data.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="task-based-shuffling-scales-poorly"&gt;
&lt;h1&gt;Task-based shuffling scales poorly&lt;/h1&gt;
&lt;p&gt;While systems like distributed databases and Apache Spark use a dedicated shuffle service to move data around the cluster, Dask builds on task-based scheduling.
Task-based systems are more flexible for general-purpose parallel computing but less suitable for all-to-all shuffling.
This results in three main issues:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Scheduler strain:&lt;/strong&gt; The scheduler hangs due to the sheer amount of tasks required for shuffling.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Full dataset materialization:&lt;/strong&gt; Workers materialize the entire dataset causing the cluster to run out of memory.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Many small operations:&lt;/strong&gt; Intermediate tasks are too small and bring down hardware performance.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Together, these issues make large-scale shuffles inefficient, causing users to over-provision clusters.
Fortunately, we can design a system to address these concerns.
Early work on this started back &lt;a class="reference external" href="https://www.coiled.io/blog/better-shuffling-in-dask-a-proof-of-concept?utm_source=dask-blog&amp;amp;amp;utm_medium=shuffling-large-data-at-constant-memory"&gt;in 2021&lt;/a&gt; and has now matured into the P2P shuffling system:&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 44)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="p2p-shuffling"&gt;
&lt;h1&gt;P2P shuffling&lt;/h1&gt;
&lt;p&gt;With release &lt;a class="reference external" href="https://distributed.dask.org/en/stable/changelog.html#v2023-2-1"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2023.2.1&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;, Dask introduces a new shuffle method called P2P, making sorts, merges, and joins run faster and in constant memory.&lt;/p&gt;
&lt;p&gt;This system is designed with three aspects, mirroring the problems listed above:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Peer-to-Peer communication:&lt;/strong&gt; Reduce the involvement of the scheduler&lt;/p&gt;
&lt;p&gt;Shuffling becomes an O(n) operation from the scheduler’s perspective, removing a key bottleneck.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="P2P shuffling uses fewer tasks than task-based shuffling." src="/images/shuffling_tasks.png" style="width: 100%;"/&gt;
&lt;/figure&gt;
&lt;ol class="arabic" start="2"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Disk by default:&lt;/strong&gt; Store data as it arrives on disk, efficiently.&lt;/p&gt;
&lt;p&gt;Dask can now shuffle datasets of arbitrary size in small memory, reducing the need to fiddle with right-sizing clusters.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Buffer network and disk with memory:&lt;/strong&gt; Avoid many small writes by buffering sensitive hardware with in-memory stores.&lt;/p&gt;
&lt;p&gt;Shuffling involves CPU, network, and disk, each of which brings its own bottlenecks when dealing with many small pieces of data.
We use memory judiciously to trade off between these bottlenecks, balancing network and disk I/O to maximize the overall throughput.&lt;/p&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In addition to these three aspects, P2P shuffling implements many minor optimizations to improve performance, and it relies on &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pyarrow&amp;gt;=7.0&lt;/span&gt;&lt;/code&gt; for efficient data handling and (de)serialization.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 69)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="results"&gt;
&lt;h1&gt;Results&lt;/h1&gt;
&lt;p&gt;To evaluate P2P shuffling, we benchmark against task-based shuffling on common workloads from our benchmark suite.
For more information on this benchmark suite, see the &lt;a class="reference external" href="https://github.com/coiled/coiled-runtime/tree/main/tests/benchmarks"&gt;GitHub repository&lt;/a&gt; or the &lt;a class="reference external" href="https://benchmarks.coiled.io/"&gt;latest test results&lt;/a&gt;.&lt;/p&gt;
&lt;section id="memory-stability"&gt;
&lt;h2&gt;Memory stability&lt;/h2&gt;
&lt;p&gt;The biggest benefit of P2P shuffling is constant memory usage.
Memory usage drops and stays constant across all workloads:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="P2P shuffling uses significantly less memory than task-based shuffling on all workloads." src="/images/p2p_benchmark_suite.png" style="width: 100%;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;P2P shuffling (blue) uses constant memory while task-based shuffling (orange) scales linearly.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;For the tested workloads, we saw up to &lt;strong&gt;10x lower memory&lt;/strong&gt;.
For even larger workloads, this gap only increases.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="performance-and-speed"&gt;
&lt;h2&gt;Performance and Speed&lt;/h2&gt;
&lt;p&gt;In the above plot, we can two performance improvements:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Faster execution:&lt;/strong&gt; Workloads run up to 45% faster.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Quicker startup:&lt;/strong&gt; Smaller graphs mean P2P shuffling starts sooner.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Comparing scheduler dashboards for P2P and task-based shuffling side-by-side
helps to understand what causes these performance gains:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Task-based shuffling shows bigger gaps in the task stream and spilling compared to P2P shuffling." src="/images/dashboard-comparison.png" style="width: 100%;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Task-based shuffling (left) shows bigger gaps in the task stream and spilling compared to P2P shuffling (right).&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Graph size:&lt;/strong&gt; 10x fewer tasks are faster to deploy and easier on the scheduler.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Controlled I/O:&lt;/strong&gt; P2P shuffling handles I/O explicitly, which avoids less performant spilling by Dask.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Fewer interruptions:&lt;/strong&gt; I/O is well balanced, so we see fewer gaps in work in the task stream.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Overall, P2P shuffling leverages our hardware far more effectively.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 112)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="changing-defaults"&gt;
&lt;h1&gt;Changing defaults&lt;/h1&gt;
&lt;p&gt;These results benefit most users, so P2P shuffling is now the default starting from release &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2023.2.1&lt;/span&gt;&lt;/code&gt;,
as long as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pyarrow&amp;gt;=7.0.0&lt;/span&gt;&lt;/code&gt; is installed. Dataframe operations like the following will benefit:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.set_index(...)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.merge(...)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.groupby(...).apply(...)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;section id="share-your-experience"&gt;
&lt;h2&gt;Share your experience&lt;/h2&gt;
&lt;p&gt;To understand how P2P performs on the vast range of workloads out there in the wild, and to ensure that making it the default was the right choice, we would love to learn about your experiences running it.
We have opened a &lt;a class="reference external" href="https://github.com/dask/distributed/discussions/7509"&gt;discussion issue&lt;/a&gt; on GitHub for feedback on this change.
Please let us know how it helps (or doesn’t).&lt;/p&gt;
&lt;/section&gt;
&lt;section id="keep-old-behavior"&gt;
&lt;h2&gt;Keep old behavior&lt;/h2&gt;
&lt;p&gt;If P2P shuffling does not work for you, you can deactivate it by setting the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dataframe.shuffle.method&lt;/span&gt;&lt;/code&gt; config value to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;quot;tasks&amp;quot;&lt;/span&gt;&lt;/code&gt; or explicitly setting a keyword argument, for example:&lt;/p&gt;
&lt;p&gt;with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;yaml&lt;/span&gt;&lt;/code&gt; config&lt;/p&gt;
&lt;div class="highlight-yaml notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nt"&gt;dataframe&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;  &lt;/span&gt;&lt;span class="nt"&gt;shuffle&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="nt"&gt;method&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="l l-Scalar l-Scalar-Plain"&gt;tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or when using a cluster manager&lt;/p&gt;
&lt;div class="highlight-python3 notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;

&lt;span class="c1"&gt;# The dataframe.shuffle.method config is available since 2023.3.1&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dataframe.shuffle.method&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;tasks&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}):&lt;/span&gt;
    &lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# many cluster managers send current dask config automatically&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For more information on deactivating P2P shuffle, see the &lt;a class="reference external" href="https://github.com/dask/distributed/discussions/7509"&gt;discussion #7509&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 152)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="what-about-arrays"&gt;
&lt;h1&gt;What about arrays?&lt;/h1&gt;
&lt;p&gt;While the original motivation was to optimize large-scale dataframe joins, the P2P system is useful for all problems requiring lots of communication between tasks.
For array workloads, this often occurs when rechunking data, such as when organizing a matrix by rows when it was stored by columns.
Similar to dataframe joins, array rechunking has been inefficient in the past, which has become such a problem that the array community built specialized tools like &lt;a class="reference external" href="https://github.com/pangeo-data/rechunker"&gt;rechunker&lt;/a&gt; to avoid it entirely.&lt;/p&gt;
&lt;p&gt;There is a &lt;a class="reference external" href="https://github.com/dask/distributed/pull/7534"&gt;naive implementation&lt;/a&gt; of array rechunking using the P2P system available for experimental use.
Benchmarking this implementation shows mixed results:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="P2P rechunking uses constant memory." src="/images/p2p_benchmark_arrays.png" style="width: 100%;"/&gt;
&lt;/figure&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;👍 &lt;strong&gt;Constant memory use:&lt;/strong&gt; As with dataframe operations, memory use is constant.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;❓ &lt;strong&gt;Variable runtime:&lt;/strong&gt; The runtime of workloads may increase with P2P.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;👎 &lt;strong&gt;Memory overhead:&lt;/strong&gt; There is a large memory overhead for many small partitions.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The constant memory use is a very promising result.
There are several ways to tackle the current limitations.
We expect this to improve as we work with collaborators from the array computing community.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 173)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="next-steps"&gt;
&lt;h1&gt;Next steps&lt;/h1&gt;
&lt;p&gt;Development on P2P shuffling is not done yet.
For the future we plan the following:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt;&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;While the early prototype of array rechunking is promising, it’s not there yet.
We plan to do the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Intelligently select which algorithm to use (task-based rechunking is better sometimes)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Work with collaborators on rechunking to improve performance&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Seek out other use cases, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;, where this might be helpful&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Failure recovery&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;Make P2P shuffling resilient to worker loss; currently, it has to restart entirely.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Performance tuning&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;Performance today is good, but not yet at peak hardware speeds.
We can improve this in a few ways:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Hiding disk I/O&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Using more memory when appropriate for smaller shuffles&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Improved batching of small operations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;More efficient serialization&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;To follow along with the development, subscribe to the &lt;a class="reference external" href="https://github.com/dask/distributed/issues/7507"&gt;tracking issue&lt;/a&gt; on GitHub.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 203)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="summary"&gt;
&lt;h1&gt;Summary&lt;/h1&gt;
&lt;p&gt;Shuffling data is a common operation in dataframe workloads.
Since &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2023.2.1&lt;/span&gt;&lt;/code&gt;, Dask defaults to P2P shuffling for distributed clusters and shuffles data faster and at constant memory.
This improvement unlocks previously un-runnable workload sizes and efficiently uses your clusters.
Finally, P2P shuffling demonstrates extending Dask to add new paradigms while leveraging the foundations of its distributed engine.&lt;/p&gt;
&lt;p&gt;Share results in &lt;a class="reference external" href="https://github.com/dask/distributed/issues/7507"&gt;this discussion thread&lt;/a&gt; or follow development at this &lt;a class="reference external" href="https://github.com/dask/distributed/issues/7507"&gt;tracking issue&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/03/15/shuffling-large-data-at-constant-memory/"/>
    <summary>This work was engineered and supported by Coiled. In particular, thanks to Florian Jetter, Gabe Joseph, Hendrik Makait, and Matt Rocklin. Original version of this post appears on blog.coiled.io</summary>
    <category term="dask" label="dask"/>
    <category term="distributed" label="distributed"/>
    <category term="p2p" label="p2p"/>
    <category term="shuffling" label="shuffling"/>
    <published>2023-03-15T00:00:00+00:00</published>
  </entry>
</feed>
