<?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 - Posted in 2024</title>
  <updated>2026-03-05T15:05:21.097561+00:00</updated>
  <link href="https://blog.dask.org"/>
  <link href="https://blog.dask.org/blog/2024/atom.xml" rel="self"/>
  <generator uri="https://ablog.readthedocs.io/" version="0.11.12">ABlog</generator>
  <entry>
    <id>https://blog.dask.org/2024/11/21/dask-detrending/</id>
    <title>Improving GroupBy.map with Dask and Xarray</title>
    <updated>2024-11-21T00:00:00+00:00</updated>
    <author>
      <name>Patrick Hoefler</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This post was originally published on the &lt;a class="reference external" href="https://xarray.dev/blog/dask-detrending"&gt;Xarray blog&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Running large-scale GroupBy-Map patterns with Xarray that are backed by &lt;a class="reference external" href="https://docs.dask.org/en/stable/array.html"&gt;Dask arrays&lt;/a&gt; is
an essential part of a lot of typical geospatial workloads. Detrending is a very common
operation where this pattern is needed.&lt;/p&gt;
&lt;p&gt;In this post, we will explore how and why this caused so many pitfalls for Xarray users in
the past and how we improved performance and scalability with a few changes to how Dask
subselects 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/2024/11/21/dask-detrending.md&lt;/span&gt;, line 20)&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-groupby-map"&gt;

&lt;p&gt;&lt;a class="reference external" href="https://docs.xarray.dev/en/stable/generated/xarray.core.groupby.DatasetGroupBy.map.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GroupBy.map&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; lets you apply a User Defined Function (UDF)
that accepts and returns Xarray objects. The UDF will receive an Xarray object (either a Dataset or a DataArray) containing Dask arrays corresponding to one single group.
&lt;a class="reference external" href="https://docs.xarray.dev/en/stable/generated/xarray.core.groupby.DatasetGroupBy.reduce.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Groupby.reduce&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; is quite similar
in that it applies a UDF, but in this case the UDF will receive the underlying Dask arrays, &lt;em&gt;not&lt;/em&gt; Xarray objects.&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/2024/11/21/dask-detrending.md&lt;/span&gt;, line 27)&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="the-application"&gt;
&lt;h1&gt;The Application&lt;/h1&gt;
&lt;p&gt;Consider a typical workflow where you want to apply a detrending step. You may want to smooth out
the data by removing the trends over time. This is a common operation in climate science
and normally looks roughly like this:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;detrending_step&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arr&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;DataArray&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;DataArray&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="c1"&gt;# important: the rolling operation is applied within a group&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;arr&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;arr&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rolling&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;30&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;min_periods&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;time.dayofyear&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;detrending_step&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We are grouping by the day of the year and then are calculating the rolling average over
30-year windows for a particular day.&lt;/p&gt;
&lt;p&gt;Our example will run on a 1 TiB array, 64 years worth of data and the following structure:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Python repr output of 1 TiB Dask array with shape (1801, 3600, 233376) split into 5460, 250 MiB chunks of (300, 300, 365)" src="/images/dask-detrending/input-array.png" style="width: 800px;"/&gt;
&lt;/figure&gt;
&lt;p&gt;The array isn’t overly huge and the chunks are reasonably sized.&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/2024/11/21/dask-detrending.md&lt;/span&gt;, line 52)&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="the-problem"&gt;
&lt;h1&gt;The Problem&lt;/h1&gt;
&lt;p&gt;The general application seems straightforward. Group by the day of the year and apply a UDF
to every group. There are a few pitfalls in this application that can make the result of
this operation unusable. Our array is sorted by time, which means that we have to pick
entries from many different areas in the array to create a single group (corresponding to a single day of the year).
Picking the same day of every year is basically a slicing operation with a step size of 365.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Schematic showing an array sorted by time, where data is selected from many different areas in the array to create a single group (corresponding to a specific day of the year)." src="/images/dask-detrending/indexing-data-selection.png" title="Data Selection Pattern" style="width: 800px;"/&gt;
&lt;/figure&gt;
&lt;p&gt;Our example has a year worth of data in a single chunk along the time axis. The general problem
exists for any workload where you have to access random entries of data. This
particular access pattern means that we have to pick one value per chunk, which is pretty
inefficient. The right side shows the individual groups that we are operating on.&lt;/p&gt;
&lt;p&gt;One of the main issues with this pattern is that Dask will create a single output chunk per time
entry, e.g. each group will consist of as many chunks as we have year.&lt;/p&gt;
&lt;p&gt;This results in a huge increase in the number of chunks:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Python repr output of a 1 TiB Dask array with nearly 2 million, 700 kiB chunks." src="/images/dask-detrending/output-array-old.png" style="width: 800px;"/&gt;
&lt;/figure&gt;
&lt;p&gt;This simple operation increases the number of chunks from 5000 to close to 2 million. Each
chunk only has a few hundred kilobytes of data. &lt;strong&gt;This is pretty bad!&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Dask computations generally scale along the number of chunks you have. Increasing the chunks by such
a large factor is catastrophic. Each follow-up operation, as simple as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a-b&lt;/span&gt;&lt;/code&gt; will create 2 million
additional tasks.&lt;/p&gt;
&lt;p&gt;The only workaround for users was to rechunk to something more sensible afterward, but it
still keeps the incredibly expensive indexing operation in the graph.&lt;/p&gt;
&lt;p&gt;Note this is the underlying problem that is &lt;a class="reference external" href="https://xarray.dev/blog/flox"&gt;solved by flox&lt;/a&gt; for aggregations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.mean()&lt;/span&gt;&lt;/code&gt;
using parallel-native algorithms to avoid the expense of indexing out each group.&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/2024/11/21/dask-detrending.md&lt;/span&gt;, line 91)&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="improvements-to-the-data-selection-algorithm"&gt;
&lt;h1&gt;Improvements to the Data Selection algorithm&lt;/h1&gt;
&lt;p&gt;The method of how Dask selected the data was objectively pretty bad.
A rewrite of the underlying algorithm enabled us to achieve a much more robust result. The new
algorithm is a lot smarter about how to pick values from each individual chunk, but most importantly,
it will try to preserve the input chunksize as closely as possible.&lt;/p&gt;
&lt;p&gt;For our initial example, it will put every group into a single chunk. This means that we will
end up with the number of chunks along the time axis being equal to the number of groups, i.e. 365.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Python repr output of a 1 TiB Dask array with 31164, 43 MiB chunks" src="/images/dask-detrending/output-array-new.png" style="width: 800px;"/&gt;
&lt;/figure&gt;
&lt;p&gt;The algorithm reduces the number of chunks from 2 million to roughly 30 thousand, which is a huge improvement
and a scale that Dask can easily handle. The graph is now much smaller, and the follow-up operations
will run a lot faster as well.&lt;/p&gt;
&lt;p&gt;This improvement will help every operation that we listed above and make the scale a lot more
reliably than before. The algorithm is used very widely across Dask and Xarray and thus, influences
many methods.&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/2024/11/21/dask-detrending.md&lt;/span&gt;, line 113)&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="what-s-next"&gt;
&lt;h1&gt;What’s next?&lt;/h1&gt;
&lt;p&gt;Xarray selects one group at a time for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby(...).map(...)&lt;/span&gt;&lt;/code&gt;, i.e. this requires one operation
per group. This will hurt scalability if the dataset has a very large number of groups, because
the computation will create a very expensive graph. There is currently an effort to implement alternative
APIs that are shuffle-based to circumvent that problem. A current PR is available &lt;a class="reference external" href="https://github.com/pydata/xarray/pull/9320"&gt;here&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The fragmentation of the output chunks by indexing is something that will hurt every workflow that is selecting data in a random
pattern. This also includes:&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;.sel&lt;/span&gt;&lt;/code&gt; if you aren’t using slices explicitly&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.isel&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;.sortby&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;groupby(...).quantile()&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;and many more.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We expect all of these workloads to be substantially improved now.&lt;/p&gt;
&lt;p&gt;Additionally, &lt;a class="reference external" href="https://docs.dask.org/en/stable/changelog.html#v2024-11-1"&gt;Dask improved a lot of things&lt;/a&gt; related to either increasing chunksizes or fragmentation
of chunks over the cycle of a workload with more improvements to come. This will help a lot of
users to get better and more reliable performance.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2024/11/21/dask-detrending/"/>
    <summary>This post was originally published on the Xarray blog.</summary>
    <category term="daskarray" label="dask array"/>
    <category term="xarray" label="xarray"/>
    <published>2024-11-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2024/05/30/dask-is-fast/</id>
    <title>Dask DataFrame is Fast Now</title>
    <updated>2024-05-30T00:00:00+00:00</updated>
    <author>
      <name>Patrick Hoefler</name>
    </author>
    <content type="html">&lt;meta content="Dask DataFrame is faster and more reliable, especially for TBs of data. This is due to engineering improvements like adding a query optimizer, integrating with Apache Arrow, and more efficient data shuffling." name="description" /&gt;
&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=dask-expr"&gt;Coiled&lt;/a&gt; and &lt;a class="reference external" href="https://www.nvidia.com/"&gt;NVIDIA&lt;/a&gt;. Thanks to &lt;a class="reference external" href="https://github.com/phofl"&gt;Patrick Hoefler&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/rjzamora"&gt;Rick Zamora&lt;/a&gt;, in particular. Original version of this post appears on &lt;a class="reference external" href="https://docs.coiled.io/blog/dask-dataframe-is-fast.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-is-fast"&gt;docs.coiled.io&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Bar chart showing a nearly 20x improvement in Dask DataFrame performance with the addition of Arrow stings, more efficient shufffling, and a query optimizer." src="/images/dask-improvement.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Performance Improvements for Dask DataFrames&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 21)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="intro"&gt;

&lt;p&gt;Dask DataFrame scales out pandas DataFrames to operate at the 100GB-100TB scale.&lt;/p&gt;
&lt;p&gt;Historically, &lt;a class="reference external" href="https://www.dask.org/"&gt;Dask&lt;/a&gt; was pretty slow compared to other tools in this space (like Spark). Due to a number of improvements focused on performance,
it’s now pretty fast (about 20x faster than before). The new implementation moved Dask from getting destroyed by
Spark on every benchmark to regularly outperforming Spark on TPC-H queries by a significant margin.&lt;/p&gt;
&lt;p&gt;Dask DataFrame workloads struggled with many things. Performance and memory usage were
commonly seen pain points, shuffling was unstable for bigger datasets, making scaling out
hard. Writing efficient code required understanding too much of the internals of Dask.&lt;/p&gt;
&lt;p&gt;The new implementation changed all of this. Things that didn’t work were completely rewritten from scratch and existing
implementations were improved upon. This puts Dask DataFrames on a solid foundation that
allows faster iteration cycles in the future.&lt;/p&gt;
&lt;p&gt;We’ll go through the three most prominent changes:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#arrow"&gt;&lt;span class="xref myst"&gt;Apache Arrow support&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#p2p"&gt;&lt;span class="xref myst"&gt;Faster joins&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#optimizer"&gt;&lt;span class="xref myst"&gt;Query optimization&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We’ll cover how these changes impact performance and make it easier to use Dask efficiently, even for users that are new to distributed computing. We’ll also discuss plans for future improvements.&lt;/p&gt;
&lt;p&gt;&lt;a name="arrow"&gt;&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 47)&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="apache-arrow-support-efficient-string-datatype"&gt;
&lt;h1&gt;1. Apache Arrow Support: Efficient String Datatype&lt;/h1&gt;
&lt;p&gt;A Dask DataFrame consists of many pandas DataFrames. Historically, pandas used NumPy for numeric data,
but Python objects for text data, which are inefficient and blow up memory usage. Operations on
object data also hold the GIL, which doesn’t matter much for pandas, but is a catastrophy for
performance with a parallel system like Dask.&lt;/p&gt;
&lt;p&gt;The pandas 2.0 release introduced support for general-purpose Arrow datatypes, so &lt;a class="reference external" href="https://docs.coiled.io/blog/pyarrow-in-pandas-and-dask.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-is-fast"&gt;Dask now uses PyArrow-backed
strings by default&lt;/a&gt;. These are &lt;em&gt;much&lt;/em&gt; better. PyArrow strings reduce memory usage by up to 80% and
unlock multi-threading for string operations. Workloads that previously
struggled with available memory now fit comfortably in much less space, and are
a lot faster because they no longer constantly spill excess data to disk.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Bar chart comparing memory usage (GB) for Dask DataFrame with and without PyArrow strings. Dask DataFrame uses up to 80% less memory with PyArrow strings." src="/images/arrow-strings-memory-usage.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Memory Usage of the Legacy DataFrames Compared with Arrow Strings&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;&lt;a name="p2p"&gt;&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/2024/05/30/dask-is-fast.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="faster-joins-with-a-new-shuffle-algorithm"&gt;
&lt;h1&gt;2. Faster Joins with a New Shuffle Algorithm&lt;/h1&gt;
&lt;p&gt;Shuffling is an essential component of distributed systems to enable sorting, joins, and complex group by operations. It is an all-to-all, network-intensive operation that’s often the most expensive component in a workflow. Dask has a new shuffling system, which greatly impacts overall performance, especially on complex, data-intensive workloads.&lt;/p&gt;
&lt;p&gt;A shuffle operation is intrinsically an all-to-all communication operation where every
input partition has to provide a tiny slice of data to every output partition. Dask was already
using it’s own task-based algorithm that managed to reduce the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;O(n&lt;/span&gt; &lt;span class="pre"&gt;*&lt;/span&gt; &lt;span class="pre"&gt;n)&lt;/span&gt;&lt;/code&gt; task
complexity to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;O(log(n)&lt;/span&gt; &lt;span class="pre"&gt;*&lt;/span&gt; &lt;span class="pre"&gt;n)&lt;/span&gt;&lt;/code&gt; where &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;n&lt;/span&gt;&lt;/code&gt; is the number of partitions. This was a drastic
reduction in the number of tasks, but the non-linear scaling ultimately did not allow Dask to process
arbitrarily large datasets.&lt;/p&gt;
&lt;p&gt;Dask introduced a new &lt;a class="reference external" href="https://docs.coiled.io/blog/shuffling-large-data-at-constant-memory.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-is-fast"&gt;P2P (peer-to-peer) shuffle method&lt;/a&gt; that reduced the task complexity to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;O(n)&lt;/span&gt;&lt;/code&gt;
which scales linearly with the size of the dataset and the size of the cluster. It also
incorporates an efficient disk integration which allows easily shuffling datasets which are much
larger than memory. The new system is extremely stable and “just works” across any scale of data.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Line plot of memory usage (GB) over time (seconds) comparing Dask DataFrame with the peer-to-peer shuffling to task-based shuffling. With P2P shuffling, Dask DataFrame memory usage remains consistently low throughout the computation." src="/images/shuffle-memory-comparison.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Memory Usage of the Legacy Shuffle Compared with P2P&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;&lt;a name="optimizer"&gt;&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 94)&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="optimizer"&gt;
&lt;h1&gt;3. Optimizer&lt;/h1&gt;
&lt;p&gt;Dask itself is lazy, which means that it registers your whole query before doing any actual work.
This is a powerful concept that enables a lot of optimizations, but historically Dask wasn’t taking advantage of this
knowledge in the past. Dask also did a bad job of hiding internal complexities and
left users on their own while navigating the difficulties of distributed computing and running
large scale queries. It made writing efficient code painful for non-experts.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org/en/stable/changelog.html#query-planning"&gt;The Dask release in March&lt;/a&gt; includes a complete re-implementation of the DataFrame API to support query optimization. This is a big deal.
The new engine centers around a query optimizer that rewrites your code to make it more efficient and
better tailored to Dask’s strengths. Let’s dive into some optimization strategies, how they make
Dask run faster and scale better.&lt;/p&gt;
&lt;p&gt;We will start with a couple general-purpose optimizations:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#column-projection"&gt;&lt;span class="xref myst"&gt;Column projection&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#filter-pushdown"&gt;&lt;span class="xref myst"&gt;Filter pushdown&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;And then dive into more specific techniques that are tailored to distributed systems generally
and Dask more specifically:&lt;/p&gt;
&lt;ol class="arabic simple" start="3"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#partition-resizing"&gt;&lt;span class="xref myst"&gt;Automatic partition resizing&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#trivial-merge"&gt;&lt;span class="xref myst"&gt;Trivial merge and join operations&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;a name="column-projection"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;section id="column-projection"&gt;
&lt;h2&gt;3.1 Column Projection&lt;/h2&gt;
&lt;p&gt;Most datasets have more columns than needed. Dropping them requires foresight (“What columns will I need for this query? 🤔”) so most people don’t think about this when loading data. This is bad for performance because carrying around lots of excess data slows everything down.
Column Projection drops columns as soon as they aren’t needed anymore. It’s a straightforward optimization, but highly beneficial.&lt;/p&gt;
&lt;p&gt;The legacy implementation always reads all columns from storage and only drops columns when explicitly specified by the user.
Simply operating on less data is a big win for performance and memory usage.&lt;/p&gt;
&lt;p&gt;The optimizer looks at the query and figures out which columns are needed for each operation.
It looks at the final step of the query and then works backwards step by
step to the data source, injecting drop operations to get rid of unnecessary columns.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram explaining how column projection works for the Dask DataFrame optimizer." src="/images/projection.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Only require a subset of columns are needed. Replace doesn't need access to all columns, so Dask drops unnecessary columns directly in the IO step.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;&lt;a name="filter-pushdown"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="filter-pushdown"&gt;
&lt;h2&gt;3.2 Filter Pushdown&lt;/h2&gt;
&lt;p&gt;Filter pushdown is another general-purpose optimization with
the same goal as column projection: operate on less data. The legacy implementation did not reorder filter operations. The new implementation executes filter operations as early as
possible while maintaining the same results.&lt;/p&gt;
&lt;p&gt;The optimizer identifies every filter in the query and looks at the previous operation to see if we
can move the filter closer to the data source. It will repeat this until it finds an operation that
can’t be switched with a filter. This is a bit harder than
column projections, because Dask has to make sure that the operations don’t change the values of the
DataFrame. For example, switching a filter and a merge operation is fine (values don’t change), but switching a filter
and a replace operation is invalid, because the values might change and rows that would previously have been filtered out now won’t be, or vice versa.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram explaining how filter pushdown (or predicate pushdown) works to reduce the amount of data being processed in the Dask DataFrame query optimizer." src="/images/filter.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Initially, the filter happens after the Dropna, but Dask can execute the filter before Dropna without changing the result. This allows Dask to push the filter into the IO step.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Additionally, if the filter is strong enough then Dask can potentially drop complete files in the IO step.
This is a best-case scenario, where an earlier filter brings a huge performance improvement and even
requires reading less data from remote storage.&lt;/p&gt;
&lt;p&gt;&lt;a name="partition-resizing"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="automatically-resizing-partitions"&gt;
&lt;h2&gt;3.3 Automatically Resizing Partitions&lt;/h2&gt;
&lt;p&gt;In addition to implementing the common optimization techniques described above, we’ve also improved a
common pain point specific to distributed systems generally and Dask users specifically: optimal partition sizes.&lt;/p&gt;
&lt;p&gt;Dask DataFrames consist of many small pandas DataFrames called &lt;em&gt;partitions&lt;/em&gt;. Often, the number of
partitions is decided for you and Dask users are advised to manually “repartition” after reducing
or expanding their data (for example by dropping columns, filtering data, or expanding with joins) (see the &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe-best-practices.html#repartition-to-reduce-overhead"&gt;Dask docs&lt;/a&gt;).
Without this extra step,
the (usually small) overhead from Dask can become a bottleneck if the pandas DataFrames
become too small, making Dask workflows painfully slow.&lt;/p&gt;
&lt;p&gt;Manually controlling the partition size is a difficult task that we, as Dask users, shouldn’t have
to worry about. It is also slow because it requires network transfer of some partitions.
Dask DataFrame now automatically does two things to help when the partitions get
too small:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Keeps the size of each partition constant, based on the ratio of data you want to compute vs.
the original file size. If, for example, you filter out 80% of the original dataset, Dask will
automatically combine the resulting smaller partitions into fewer, larger partitions.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Combines too-small partitions into larger partitions, based on an absolute minimum
(default is 75 MB). If, for example, your original dataset is split into many tiny files,
Dask will automatically combine them.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram representing how column selection can reduce the size of a partition when reading parquet data." src="/images/automatic_repartitioning_1.png" style="width: 800px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Select two columns that take up 40 MB of memory out of the 200 MB from the whole parquet file.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The optimizer will look at the number of columns and the size of the data within those. It
calculates a ratio that is used to combine multiple files into one partition.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram showing how automatic repartitioning works for Dask DataFrame." src="/images/automatic_repartitioning_2.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;The ratio of 40/200 results in combining five files into a single partition.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;This step is currently limited to IO operations (like reading in a parquet dataset), but we plan
to extend it to other operations that allow cheaply combining partitions.&lt;/p&gt;
&lt;p&gt;&lt;a name="trivial-merge"&gt;&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;section id="trivial-merge-and-join-operations"&gt;
&lt;h2&gt;3.4 Trivial Merge and Join Operations&lt;/h2&gt;
&lt;p&gt;Merge and join operations are typically cheap on a single machine with pandas but expensive in a
distributed setting. Merging data in shared memory is cheap, while merging data across a network is quite slow,
due to the shuffle operations explained earlier.&lt;/p&gt;
&lt;p&gt;This is one of the most expensive operations in a distributed system. The legacy implementation triggered
a network transfer of both input DataFrames for every merge operation. This is sometimes necessary, but very
expensive.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram representing how an expensive shuffle operation is avoided automatically if two DataFrames are already aligned before joining." src="/images/avoiding-shuffles.png" style="width: 800px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Both joins are performed on the same column. The left DataFrame is already properly partitioned after the first join, so Dask can avoid shuffling again with the new implementation.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The optimizer will determine when shuffling is necessary versus when
a trivial join is sufficient because the data is already aligned properly. This can make individual merges
an order of magnitude faster. This also applies to other operations that normally require a shuffle
like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby().apply()&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Dask merges used to be inefficient, which caused long runtimes. The optimizer fixes this for
the trivial case where these operations happen after each other, but the technique isn’t very
advanced yet. There is still a lot of potential for improvement.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Diagram showing how the query optimizer for Dask DataFrame automatically shuffles data earlier to make a groupby aggregation more efficient." src="/images/avoiding-shuffles-advanced.png" style="width: 800px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;The current implementation shuffles both branches that originate from the same table. Injecting a shuffle node further up avoids one of the expensive operations.&lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;The optimizer will look at the expression and inject shuffle nodes where necessary to avoid
unnecessary shuffles.&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 250)&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="how-do-the-improvements-stack-up-compared-to-the-legacy-implementation"&gt;
&lt;h1&gt;How do the improvements stack up compared to the legacy implementation?&lt;/h1&gt;
&lt;p&gt;Dask is now 20x faster than before. This improvement applies to the entire
DataFrame API (not just isolated components), with no known
performance regressions. Dask now runs workloads that were impossible to
complete in an acceptable timeframe before. This performance boost is due to many
improvements all layered on top of each other. It’s not about doing one thing
especially well, but about doing nothing especially poorly.&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Bar chart showing a nearly 20x improvement in Dask DataFrame performance with the addition of Arrow stings, more efficient shufffling, and a query optimizer." src="/images/dask-improvement.png" style="width: 600px;"/&gt;
&lt;figcaption&gt;
&lt;em&gt;Performance Improvements on Query 3 of the TPC-H Benchmarks from &lt;a class="reference external" href="https://github.com/coiled/benchmarks/tree/main/tests/tpch"&gt;https://github.com/coiled/benchmarks/tree/main/tests/tpch&lt;/a&gt; &lt;/em&gt;
&lt;/figcaption&gt;
&lt;/figure&gt;
&lt;p&gt;Performance, while the most enticing improvement, is not the only thing that got better. The
optimizer hides a lot of complexity from the user and makes the transition from pandas to Dask a
lot easier because it’s now much more difficult to write poorly performing code.
The whole system is more robust.&lt;/p&gt;
&lt;p&gt;The new architecture of the API is a lot easier to work with as well. The legacy implementation leaked
a lot of internal complexities into high-level API implementations, making changes cumbersome. Improvements
are almost trivial to add now.&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 275)&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="what-s-to-come"&gt;
&lt;h1&gt;What’s to come?&lt;/h1&gt;
&lt;p&gt;Dask DataFrame changed a lot over the last 18 months. The legacy API was often difficult to work with and
struggled with scaling out. The new implementation dropped things that didn’t work and
improved existing implementations. The heavy lifting is finished now, which allows for
faster iteration cycles to improve upon the status quo. Incremental improvements are now
trivial to add.&lt;/p&gt;
&lt;p&gt;A few things that are on the immediate roadmap:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Auto repartitioning:&lt;/strong&gt; this is partially implemented, but there is more potential to choose a more
efficient partition size during optimization.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Faster Joins:&lt;/strong&gt; there’s still lots of fine-tuning to be done here.
For example, there is a PR in flight with a 30-40% improvement.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Join Reordering:&lt;/strong&gt; Dask doesn’t do this yet, but it’s on the immediate roadmap&lt;/p&gt;&lt;/li&gt;
&lt;/ul&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/2024/05/30/dask-is-fast.md&lt;/span&gt;, line 291)&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="learn-more"&gt;
&lt;h1&gt;Learn more&lt;/h1&gt;
&lt;p&gt;This article focuses on a number of improvements to Dask DataFrame and how much faster and more reliable it is as a result. If you’re choosing between Dask and other popular DataFrame tools, you might also consider:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://docs.coiled.io/blog/tpch.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-is-fast"&gt;DataFrames at Scale Comparison: TPC-H&lt;/a&gt; which compares Dask, Spark, Polars, and DuckDB performance on datasets ranging from 10 GB to 10 TB both locally and on the cloud&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2024/05/30/dask-is-fast/"/>
    <summary>This work was engineered and supported by Coiled and NVIDIA. Thanks to Patrick Hoefler and Rick Zamora, in particular. Original version of this post appears on docs.coiled.io</summary>
    <category term="dask" label="dask"/>
    <category term="dataframe" label="dataframe"/>
    <category term="queryoptimizer" label="query optimizer"/>
    <published>2024-05-30T00:00:00+00:00</published>
  </entry>
</feed>
