<?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 query optimizer</title>
  <updated>2026-03-05T15:05:27.403325+00:00</updated>
  <link href="https://blog.dask.org"/>
  <link href="https://blog.dask.org/blog/tag/query-optimizer/atom.xml" rel="self"/>
  <generator uri="https://ablog.readthedocs.io/" version="0.11.12">ABlog</generator>
  <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>
  <entry>
    <id>https://blog.dask.org/2023/08/25/dask-expr-introduction/</id>
    <title>High Level Query Optimization in Dask</title>
    <updated>2023-08-25T00:00:00+00:00</updated>
    <author>
      <name>Patrick Hoefler</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=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://blog.coiled.io/blog/dask-expr-introduction.html?utm_source=dask-blog&amp;amp;amp;utm_medium=dask-expr"&gt;blog.coiled.io&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="Expression tree encoded by dask-expr" src="/images/dask_expr/dask-expr-introduction-title.png" style="width: 70%;"/&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/2023/08/25/dask-expr-introduction.md&lt;/span&gt;, line 16)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="introduction"&gt;

&lt;p&gt;Dask DataFrame doesn’t currently optimize your code for you (like Spark or a SQL database would).
This means that users waste a lot of computation. Let’s look at a common example
which looks ok at first glance, but is actually pretty inefficient.&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;

&lt;span class="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;s3://coiled-datasets/uber-lyft-tlc/&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="c1"&gt;# unnecessarily reads all rows and columns&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;hvfhs_license_num&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;HV0003&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;    &lt;span class="c1"&gt;# could push the filter into the read parquet call&lt;/span&gt;
    &lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;                                &lt;span class="c1"&gt;# should read only necessary columns&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We can make this run much faster with a few simple steps:&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="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;s3://coiled-datasets/uber-lyft-tlc/&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;filters&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;hvfhs_license_num&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;==&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;HV0003&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)],&lt;/span&gt;
    &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;tips&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Currently, Dask DataFrame wouldn’t optimize this for you, but a new effort that is built around
logical query planning in Dask DataFrame will do this for you. This article introduces some of
those changes that are developed in &lt;a class="reference external" href="https://github.com/dask-contrib/dask-expr"&gt;dask-expr&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;You can install and try &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; with:&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="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&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;expr&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We are using the &lt;a class="reference external" href="https://www.nyc.gov/site/tlc/about/tlc-trip-record-data.page"&gt;NYC Taxi&lt;/a&gt;
dataset in this post.&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/08/25/dask-expr-introduction.md&lt;/span&gt;, line 60)&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="dask-expressions"&gt;
&lt;h1&gt;Dask Expressions&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask-contrib/dask-expr"&gt;Dask expressions&lt;/a&gt; provides a logical query planning layer on
top of Dask DataFrames. Let’s look at our initial example and investigate how we can improve the efficiency
through a query optimization layer. As noted initially, there are a couple of things that aren’t ideal:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;We are reading all rows into memory instead of filtering while reading the parquet files.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We are reading all columns into memory instead of only the columns that are necessary.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We are applying the filter and the aggregation onto all columns instead of only &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The query optimization layer from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; can help us with that. It will look at this expression
and determine that not all rows are needed. An intermediate layer will transpile the filter into
a valid filter-expression for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_parquet&lt;/span&gt;&lt;/code&gt;:&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="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;s3://coiled-datasets/uber-lyft-tlc/&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;filters&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;hvfhs_license_num&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;==&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;HV0003&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)],&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This still reads every column into memory and will compute the sum of every numeric column. The
next optimization step is to push the column selection into the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_parquet&lt;/span&gt;&lt;/code&gt; call as well.&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="n"&gt;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_parquet&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="s2"&gt;&amp;quot;s3://coiled-datasets/uber-lyft-tlc/&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;tips&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
    &lt;span class="n"&gt;filters&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;hvfhs_license_num&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;==&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;HV0003&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)],&lt;/span&gt;
&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is a basic example that you could rewrite by hand. Use cases that are closer to real
workflows might potentially have hundreds of columns, which makes rewriting them very strenuous
if you need a non-trivial subset of them.&lt;/p&gt;
&lt;p&gt;Let’s take a look at how we can achieve this. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; records the expression as given by the
user in an expression tree:&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="n"&gt;result&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;pprint&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;Projection&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tips&amp;#39;&lt;/span&gt;
  &lt;span class="n"&gt;Sum&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;
    &lt;span class="n"&gt;Filter&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
      &lt;span class="n"&gt;ReadParquet&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://coiled-datasets/uber-lyft-tlc/&amp;#39;&lt;/span&gt;
      &lt;span class="n"&gt;EQ&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;right&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;HV0003&amp;#39;&lt;/span&gt;
        &lt;span class="n"&gt;Projection&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;hvfhs_license_num&amp;#39;&lt;/span&gt;
          &lt;span class="n"&gt;ReadParquet&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://coiled-datasets/uber-lyft-tlc/&amp;#39;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This tree represents the expression as is. We can observe that we would read the whole dataset into
memory before we apply the projections and filters. One observation of note: It seems like we
are reading the dataset twice, but Dask is able to fuse tasks that are doing the same to avoid
computing these things twice. Let’s reorder the expression to make it more efficient:&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="n"&gt;result&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;simplify&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;pprint&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;Sum&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;numeric_only&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;
  &lt;span class="n"&gt;ReadParquet&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;s3://coiled-datasets/uber-lyft-tlc/&amp;#39;&lt;/span&gt;
               &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tips&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
               &lt;span class="n"&gt;filters&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;hvfhs_license_num&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;==&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;HV0003&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This looks quite a bit simpler. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; reordered the query and pushed the filter and the column
projection into the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_parquet&lt;/span&gt;&lt;/code&gt; call. We were able to remove quite a few steps from our expression
tree and make the remaining expressions more efficient as well. This represents the steps that
we did manually in the beginning. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; performs these steps for arbitrary many columns without
increasing the burden on the developers.&lt;/p&gt;
&lt;p&gt;These are only the two most common and easy to illustrate optimization techniques from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt;.
Some other useful optimizations are already available:&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;len(...)&lt;/span&gt;&lt;/code&gt; will only use the Index to compute the length; additionally we can ignore many operations
that won’t change the shape of a DataFrame, like a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;replace&lt;/span&gt;&lt;/code&gt; call.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;set_index&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sort_values&lt;/span&gt;&lt;/code&gt; won’t eagerly trigger computations.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Better informed selection of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;merge&lt;/span&gt;&lt;/code&gt; algorithms.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;…&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We are still adding more optimization techniques to make Dask DataFrame queries more efficient.&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/08/25/dask-expr-introduction.md&lt;/span&gt;, line 145)&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="try-it-out"&gt;
&lt;h1&gt;Try it out&lt;/h1&gt;
&lt;p&gt;The project is in a state where interested users should try it out. We published a couple of
releases. The API covers a big chunk of the Dask DataFrame API, and we keep adding more.
We have already observed very impressive performance improvements for workflows that would benefit
from query optimization. Memory usage is down for these workflows as well.&lt;/p&gt;
&lt;p&gt;We are very much looking for feedback and potential avenues to improve the library. Please give it
a shot and share your experience with us.&lt;/p&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-expr&lt;/span&gt;&lt;/code&gt; is not integrated into the main Dask DataFrame implementation yet. You can install it
with:&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="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&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;expr&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The API is very similar to what Dask DataFrame provides. It exposes mostly the same methods as
Dask DataFrame does. You can use the same methods in most cases.&lt;/p&gt;
&lt;div class="highlight-default 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_expr&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can find a list of supported operations in the
&lt;a class="reference external" href="https://github.com/dask-contrib/dask-expr#api-coverage"&gt;Readme&lt;/a&gt;. This project is still very much
in progress. The API might change without warning. We are aiming for weekly releases to push new
features out as fast as possible.&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/08/25/dask-expr-introduction.md&lt;/span&gt;, line 174)&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="why-are-we-adding-this-now"&gt;
&lt;h1&gt;Why are we adding this now?&lt;/h1&gt;
&lt;p&gt;Historically, Dask focused on flexibility and smart scheduling instead of query optimization.
The distributed scheduler built into Dask uses sophisticated algorithms to ensure ideal scheduling
of individual tasks. It tries to ensure that your resources are utilized as efficient as possible.
The graph construction process enables Dask users to build very
flexible and complicated graphs that reach beyond SQL operations. The flexibility that is provided
by the &lt;a class="reference external" href="https://docs.dask.org/en/latest/futures.html"&gt;Dask futures API&lt;/a&gt; requires very intelligent
algorithms, but it enables users to build highly sophisticated graphs. The following picture shows
the graph for a credit risk model:&lt;/p&gt;
&lt;a href="/images/dask_expr/graph_credit_risk_model.png"&gt;
&lt;img src="/images/dask_expr/graph_credit_risk_model.png"
     width="70%"
     alt="Computation graph representing a credit risk model"&gt;&lt;/a&gt;
&lt;p&gt;The nature of the powerful scheduler and the physical optimizations enables us to build very
complicated programs that will then run efficiently. Unfortunately, the nature of these optimizations
does not enable us to avoid scheduling work that is not necessary. This is where the current effort
to build high level query optimization into Dask comes in.&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/08/25/dask-expr-introduction.md&lt;/span&gt;, line 195)&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="conclusion"&gt;
&lt;h1&gt;Conclusion&lt;/h1&gt;
&lt;p&gt;Dask comes with a very smart distributed scheduler but without much logical query planning. This
is something we are rectifying now through building a high level query optimizer into Dask
DataFrame. We expect to improve performance and reduce memory usage for an average Dask workflow.&lt;/p&gt;
&lt;p&gt;This API is read for interested users to play around with. It covers a good chunk of the DataFrame
API. The library is under active development, we expect to add many more interesting things over
the coming weeks and months.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/08/25/dask-expr-introduction/"/>
    <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 blog.coiled.io</summary>
    <category term="dask" label="dask"/>
    <category term="performance" label="performance"/>
    <category term="queryoptimizer" label="query optimizer"/>
    <published>2023-08-25T00:00:00+00:00</published>
  </entry>
</feed>
