<?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 performance</title>
  <updated>2026-03-05T15:05:26.695014+00:00</updated>
  <link href="https://blog.dask.org"/>
  <link href="https://blog.dask.org/blog/tag/performance/atom.xml" rel="self"/>
  <generator uri="https://ablog.readthedocs.io/" version="0.11.12">ABlog</generator>
  <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>
  <entry>
    <id>https://blog.dask.org/2021/11/02/choosing-dask-chunk-sizes/</id>
    <title>Choosing good chunk sizes in Dask</title>
    <updated>2021-11-02T00:00:00+00:00</updated>
    <author>
      <name>Genevieve Buckley</name>
    </author>
    <content type="html">&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/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="summary"&gt;

&lt;p&gt;Confused about choosing &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-best-practices.html#select-a-good-chunk-size"&gt;a good chunk size&lt;/a&gt; for Dask arrays?&lt;/p&gt;
&lt;p&gt;Array chunks can’t be too big (we’ll run out of memory), or too small (the overhead introduced by Dask becomes overwhelming). So how can we get it right?&lt;/p&gt;
&lt;p&gt;It’s a two step process:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;First, start by choosing a chunk size similar to data you know can be processed entirely within memory (i.e. without Dask), using these &lt;a class="reference internal" href="#rough-rules-of-thumb"&gt;&lt;span class="xref myst"&gt;rough rules of thumb&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Then, watch the Dask dashboard task stream and worker memory plots, and adjust if needed. &lt;a class="reference internal" href="#what-to-watch-for-on-the-dashboard"&gt;&lt;span class="xref myst"&gt;Here are the signs to watch out for&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&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/2021/11/02/choosing-dask-chunk-sizes.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&gt;
&lt;section id="contents"&gt;
&lt;h1&gt;Contents&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#what-are-dask-array-chunks"&gt;&lt;span class="xref myst"&gt;What are Dask array chunks?&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#too-small-is-a-problemg"&gt;&lt;span class="xref myst"&gt;Too small is a problem&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#too-big-is-also-a-problem"&gt;&lt;span class="xref myst"&gt;Too big is also a problem&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#choosing-an-initial-chunk-size"&gt;&lt;span class="xref myst"&gt;Choosing an initial chunk size&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#rough-rules-of-thumb"&gt;&lt;span class="xref myst"&gt;Rough rules of thumb&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#chunks-should-be-aligned-with-array-storage-on-disk"&gt;&lt;span class="xref myst"&gt;Chunks should be aligned with array storage on disk&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#using-the-dask-dashboard"&gt;&lt;span class="xref myst"&gt;Using the Dask dashboard&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#what-to-watch-for-on-the-dashboard"&gt;&lt;span class="xref myst"&gt;What to watch for on the dashboard
&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#rechunking-arrays"&gt;&lt;span class="xref myst"&gt;Rechunking arrays&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#unmanaged-memory"&gt;&lt;span class="xref myst"&gt;Unmanaged memory&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#thanks-for-reading"&gt;&lt;span class="xref myst"&gt;Thanks for reading&lt;/span&gt;&lt;/a&gt;&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/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 35)&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-are-dask-array-chunks"&gt;
&lt;h1&gt;What are Dask array chunks?&lt;/h1&gt;
&lt;p&gt;Dask arrays are big structures, made out of many small chunks.
Typically, each small chunk is an individual &lt;a class="reference external" href="https://numpy.org/"&gt;numpy array&lt;/a&gt;, and they are arranged together to make a much larger Dask array.&lt;/p&gt;
&lt;img src="https://raw.githubusercontent.com/dask/dask/ac01ddc9074365e40d888f80f5bcd955ba01e872/docs/source/images/dask-array-black-text.svg" alt="Diagram: Dask array chunks" width="400" height="300" /&gt;
&lt;p&gt;You can find more information about Dask array chunks on this page of the documentation: &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-chunks.html"&gt;https://docs.dask.org/en/latest/array-chunks.html&lt;/a&gt;&lt;/p&gt;
&lt;section id="how-do-i-know-what-chunks-my-array-has"&gt;
&lt;h2&gt;How do I know what chunks my array has?&lt;/h2&gt;
&lt;p&gt;If you have a Dask array, you can use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chunksize&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chunks&lt;/span&gt;&lt;/code&gt; attribues to see information about the chunks. You can also visualize this with the Dask array HTML representation.&lt;/p&gt;
&lt;img src="/images/choosing-good-chunk-sizes/examine-dask-array-chunks.png" alt="Visualizating Dask array chunks with the HTML repr" width="611" height="523" /&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;arr.chunksize&lt;/span&gt;&lt;/code&gt; shows the largest chunk size. For arrays where you expect roughly uniform chunk sizes, this is a good way to summarize chunk size information.&lt;/p&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;arr.chunks&lt;/span&gt;&lt;/code&gt; shows fully explicit sizes of all chunks along all dimensions within the Dask array (see &lt;a class="reference external" href="https://docs.dask.org/en/stable/array-chunks.html#specifying-chunk-shapes"&gt;item 3 here&lt;/a&gt;). This is more verbose, and is a good choice with arrays that have irregular chunks.&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/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 54)&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="too-small-is-a-problem"&gt;
&lt;h1&gt;Too small is a problem&lt;/h1&gt;
&lt;p&gt;If array chunks are too small, it’s inefficient. Why is this?&lt;/p&gt;
&lt;p&gt;Using Dask introduces some amount of overhead for each task in your computation.
This overhead is the reason the Dask best practices advise you to &lt;a class="reference external" href="https://docs.dask.org/en/latest/best-practices.html#avoid-very-large-graphs"&gt;avoid too-large graphs&lt;/a&gt;.
This is because if the amount of actual work done by each task is very tiny, then the percentage of overhead time vs useful work time is not good.&lt;/p&gt;
&lt;p&gt;Typically, the Dask scheduler takes 1 millisecond to coordinate a single task. That means we want the computation time for each task to be comparitively larger, eg: seconds instead of milliseconds.&lt;/p&gt;
&lt;p&gt;It might be hard to understand this intuitively, so here’s an analogy. Let’s imagine we’re building a house. It’s a pretty big job, and if there were only one worker it would take much too long to build.
So we have a team of workers and a site foreman. The site foreman is equivalent to the Dask scheduler: their job is to tell the workers what tasks they need to do.&lt;/p&gt;
&lt;p&gt;Say we have a big pile of bricks to build a wall, sitting in the corner of the building site.
If the foreman (the Dask scheduler) tells workers to go and fetch a single brick at a time, then bring each one to where the wall is being built, you can see how this is going to be very slow and inefficient! The workers are spending most of their time moving between the wall and the pile of bricks. Much less time is going towards doing the actual work of mortaring bricks onto the wall.&lt;/p&gt;
&lt;p&gt;Instead, we can do this in a smarter way. The foreman (Dask scheduler) can tell the workers to go and bring one full wheelbarrow load of bricks back each time. Now workers are spending much less time moving between the wall and the pile of bricks, and the wall will be finished much quicker.&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/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 72)&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="too-big-is-also-a-problem"&gt;
&lt;h1&gt;Too big is also a problem&lt;/h1&gt;
&lt;p&gt;If the Dask array chunks are too big, this is also bad. Why?
Chunks that are too large are bad because then you are likely to run out of working memory.
You may see out of memory errors happening, or you might see performance decrease substantially as data spills to disk.&lt;/p&gt;
&lt;p&gt;When too much data is loaded in memory on too few workers, Dask will try to spill data to disk instead of crashing.
Spilling data to disk makes things run very slowly, because all the extra read/write operations to disk. Things don’t just get a little bit slower, they get a LOT slower, so it’s smart to watch out for this.&lt;/p&gt;
&lt;p&gt;To watch out for this, look at the &lt;strong&gt;worker memory plot&lt;/strong&gt; on the Dask dashboard.
Orange bars are a warning you are close to the limit, and gray means data is being spilled to disk - not good!
For more tips, see the section on &lt;a class="reference internal" href="#using-the-Dask-dashboard"&gt;&lt;span class="xref myst"&gt;using the Dask dashboard&lt;/span&gt;&lt;/a&gt; below.&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/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 85)&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="choosing-an-initial-chunk-size"&gt;
&lt;h1&gt;Choosing an initial chunk size&lt;/h1&gt;
&lt;section id="rough-rules-of-thumb"&gt;
&lt;h2&gt;Rough rules of thumb&lt;/h2&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;If you already created a prototype, which may not involve Dask at all, using a small subset of the data you intend to process, you’ll have a clear idea of what size of data can be processed easily for this workflow. You can use this knowledge to choose similar sized chunks in Dask.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Some people have observed that chunk sizes below 1MB are almost always bad. Chunk size between 100MB and 1GB are generally good, going over 1 or 2GB means you have a really big dataset and/or a lot of memory available per core,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Upper bound: Avoid too large task graphs. More than 10,000 or 100,000 chunks may start to perform poorly.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Lower bound: To get the advantage of parallelization, you need the number of chunks to at least equal the number of worker cores available (or better, the number of worker cores times 2). Otherwise, some workers will stay idle.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The time taken to compute each task should be much larger than the time needed to schedule the task. The Dask scheduler takes roughly 1 millisecond to coordinate a single task, so a good task computation time would be measured in seconds (not milliseconds).&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="chunks-should-be-aligned-with-array-storage-on-disk"&gt;
&lt;h2&gt;Chunks should be aligned with array storage on disk&lt;/h2&gt;
&lt;p&gt;If you are reading data from disk, the storage structure will inform what shape your Dask array chunks should be. For best performance, choose chunks that are well aligned with the way data is stored.&lt;/p&gt;
&lt;p&gt;From the Dask best practices on how to &lt;a class="reference external" href="https://docs.dask.org/en/stable/array-best-practices.html#orient-your-chunks"&gt;orient your chunks&lt;/a&gt;:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;When reading data you should align your chunks with your storage format. Most array storage formats store data in chunks themselves. If your Dask array chunks aren’t multiples of these chunk shapes then you will have to read the same data repeatedly, which can be expensive. Note though that often storage formats choose chunk sizes that are much smaller than is ideal for Dask, closer to 1MB than 100MB. In these cases you should choose a Dask chunk size that aligns with the storage chunk size and that every Dask chunk dimension is a multiple of the storage chunk dimension.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Some examples of data storage structures on disk include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;A HDF5 or &lt;a class="reference external" href="https://zarr.readthedocs.io/en/stable/api/core.html"&gt;Zarr array&lt;/a&gt;. The size and shape of chunks/blocks stored on disk should align well with the Dask array chunks you select.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A folder full of tiff files. You might decide that each tiff file should become a single chunk in the Dask array (or that multiple tiff files should be grouped into a single chunk).&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/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 108)&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="using-the-dask-dashboard"&gt;
&lt;h1&gt;Using the Dask dashboard&lt;/h1&gt;
&lt;p&gt;The second part of choosing a good chunk size is monitoring the Dask dashboard to see if you need to make any adjustments.&lt;/p&gt;
&lt;p&gt;If you’re not very familiar with the Dask dashboard, or you just sometimes forget where to find certain dashboard plots (like the worker memory plot), then you’ll probably enjoy these quick video tutorials:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://www.youtube.com/watch?v=N_GqzcuGLCY"&gt;Intro to the Dask dashboard (18 minute video)&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://www.youtube.com/watch?v=EX_voquHdk0"&gt;Dask Jupyterlab extension (6 minute video)&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://distributed.dask.org/en/latest/diagnosing-performance.html"&gt;Dask dashboard documentation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We recommend always having the dashboard up when you’re working with Dask.
It’s a fantastic way to get a sense of what’s working well, or poorly, so you can make adjustments.&lt;/p&gt;
&lt;section id="what-to-watch-for-on-the-dashboard"&gt;
&lt;h2&gt;What to watch for on the dashboard&lt;/h2&gt;
&lt;p&gt;Bad signs to watch out for include:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Lots of white space in the task stream plot is a bad sign. White space means nothing is happening. Chunks may be too small.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Lots and lots of red in the task stream plot is a bad sign. Red means worker communication. Dask workers need some communication, but if they are doing almost nothing except communication then there is not much productive work going on.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;On the worker memory plot, watch out for orange bars which are a sign you are getting close to the memory limit. Chunks may be too big.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;On the worker memory plot, watch out for grey bars which mean data is being spilled to disk. Chunks may be too big.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Here is an example of the Dask dashboard during a good computation (&lt;a class="reference external" href="https://youtu.be/N_GqzcuGLCY?t=372"&gt;time 6:12 in this video&lt;/a&gt;).
&lt;img alt="Visualizating Dask array chunks with the HTML repr" src="https://blog.dask.org/_images/good-dask-dashboard.png" /&gt;&lt;/p&gt;
&lt;p&gt;For comparison, here is an example of the Dask dashboard during a bad computation (&lt;a class="reference external" href="https://youtu.be/N_GqzcuGLCY?t=417"&gt;time 6:57 in this video&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;In this example, it’s inefficient because the chunks are much too small, so we see a lot of white space and red worker communication in the task stream plot.
&lt;img alt="Visualizating Dask array chunks with the HTML repr" src="https://blog.dask.org/_images/bad-dask-dashboard-zoomedin.png" /&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/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 138)&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="rechunking-arrays"&gt;
&lt;h1&gt;Rechunking arrays&lt;/h1&gt;
&lt;p&gt;If you need to change the chunking of a Dask array in the middle of a computation, you can do that with the &lt;a class="reference external" href="https://docs.dask.org/en/latest/generated/dask.array.rechunk.html"&gt;rechunk&lt;/a&gt; method.&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;rechunked_array&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;original_array&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rechunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;new_shape&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;strong&gt;Warning:&lt;/strong&gt; Rechunking Dask arrays comes at a cost.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;The Dask graph must be rearranged to accomodate the new chunk structure. This happens immediately, and will block any other interaction with python until Dask has rearranged the task graph.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;This also inserts new tasks into the Dask graph. At compute time, there are now more tasks to execute.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;For these reasons, it is best to choose a good initial chunk size and avoid rechunking.&lt;/p&gt;
&lt;p&gt;However, sometimes the data is stored on disk is not well aligned and rechunking may be necessary.
For an example of this, here is Draga Doncila Pop &lt;a class="reference external" href="https://youtu.be/10Ws59NGDaE?t=833"&gt;talking about chunk alignment&lt;/a&gt; with satellite image data.&lt;/p&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://rechunker.readthedocs.io/en/latest/"&gt;rechunker&lt;/a&gt; library can be useful in these situations:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Rechunker takes an input array (or group of arrays) stored in a persistent storage device (such as a filesystem or a cloud storage bucket) and writes out an array (or group of arrays) with the same data, but different chunking scheme, to a new location. Rechunker is designed to be used within a parallel execution framework such as Dask.&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&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/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 160)&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="unmanaged-memory"&gt;
&lt;h1&gt;Unmanaged memory&lt;/h1&gt;
&lt;p&gt;Last, remember that you don’t only need to consider the size of the array chunks in memory, but also the working memory consumed by your analysis functions. Sometimes that is called “unmanaged memory” in Dask.&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;“Unmanaged memory is RAM that the Dask scheduler is not directly aware of and which can cause workers to run out of memory and cause computations to hang and crash.” – Guido Imperiale&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;Here are some tips for handling unmanaged memory:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://coiled.io/blog/tackling-unmanaged-memory-with-dask/"&gt;Tackling unmanaged memory with Dask (Coiled blogpost)&lt;/a&gt; by Guido Imperiale&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://youtu.be/nwR6iGR0mb0"&gt;Handle Unmanaged Memory in Dask (8 minute video)&lt;/a&gt;&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/2021/11/02/choosing-dask-chunk-sizes.md&lt;/span&gt;, line 171)&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="thanks-for-reading"&gt;
&lt;h1&gt;Thanks for reading&lt;/h1&gt;
&lt;p&gt;We hope this was helpful figuring out how to choose good chunk sizes for Dask. This blogpost was inspired by &lt;a class="reference external" href="https://twitter.com/DataNerdery/status/1424953376043790341"&gt;this twitter thread&lt;/a&gt;. If you’d like to follow Dask on Twitter, you can do that at &lt;a class="reference external" href="https://twitter.com/dask_dev"&gt;https://twitter.com/dask_dev&lt;/a&gt;&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/11/02/choosing-dask-chunk-sizes/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="performance" label="performance"/>
    <published>2021-11-02T00:00:00+00:00</published>
  </entry>
</feed>
