<?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 dataframe</title>
  <updated>2026-03-05T15:05:26.195095+00:00</updated>
  <link href="https://blog.dask.org"/>
  <link href="https://blog.dask.org/blog/tag/dataframe/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/04/14/scheduler-environment-requirements/</id>
    <title>Do you need consistent environments between the client, scheduler and workers?</title>
    <updated>2023-04-14T00:00:00+00:00</updated>
    <author>
      <name>Florian Jetter</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;Update May 3rd 2023: &lt;a class="reference external" href="https://github.com/dask/dask-blog/pull/166"&gt;Clarify GPU recommendations&lt;/a&gt;.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;With the release &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2023.4.0&lt;/span&gt;&lt;/code&gt; of dask and distributed we are making a change which may require the Dask scheduler to have consistent software and hardware capabilities as the client and workers.&lt;/p&gt;
&lt;p&gt;It has always been recommended that your client and workers have a consistent software and hardware environment so that data structures and dependencies can be pickled and passed between them. However recent changes to the Dask scheduler mean that we now also require your scheduler to have the same consistent environment as everything else.&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/04/14/scheduler-environment-requirements.md&lt;/span&gt;, line 15)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="what-does-this-mean-for-me"&gt;

&lt;p&gt;For most users, this change should go unnoticed as it is common to run all Dask components in the same conda environment or docker image and typically on homogenous machines.&lt;/p&gt;
&lt;p&gt;However, for folks who may have optimized their schedulers to use cut-down environments, or for users with specialized hardware such as GPUs available on their client/workers but not the scheduler there may be some impact.&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/04/14/scheduler-environment-requirements.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&gt;
&lt;section id="what-will-the-impact-be"&gt;
&lt;h1&gt;What will the impact be?&lt;/h1&gt;
&lt;p&gt;If you run into errors such as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;&amp;quot;RuntimeError:&lt;/span&gt; &lt;span class="pre"&gt;Error&lt;/span&gt; &lt;span class="pre"&gt;during&lt;/span&gt; &lt;span class="pre"&gt;deserialization&lt;/span&gt; &lt;span class="pre"&gt;of&lt;/span&gt; &lt;span class="pre"&gt;the&lt;/span&gt; &lt;span class="pre"&gt;task&lt;/span&gt; &lt;span class="pre"&gt;graph.&lt;/span&gt; &lt;span class="pre"&gt;This&lt;/span&gt; &lt;span class="pre"&gt;frequently&lt;/span&gt; &lt;span class="pre"&gt;occurs&lt;/span&gt; &lt;span class="pre"&gt;if&lt;/span&gt; &lt;span class="pre"&gt;the&lt;/span&gt; &lt;span class="pre"&gt;Scheduler&lt;/span&gt; &lt;span class="pre"&gt;and&lt;/span&gt; &lt;span class="pre"&gt;Client&lt;/span&gt; &lt;span class="pre"&gt;have&lt;/span&gt; &lt;span class="pre"&gt;different&lt;/span&gt; &lt;span class="pre"&gt;environments.&amp;quot;&lt;/span&gt;&lt;/code&gt; please ensure your software environment is consistent between your client, scheduler and workers.&lt;/p&gt;
&lt;p&gt;If you are passing GPU objects between the client and workers we now recommend that your scheduler has a GPU too. This recommendation is just so that GPU-backed objects contained in Dask graphs can be deserialized on the scheduler if necessary. Typically the GPU available to the scheduler doesn’t need to be as powerful as long as it has &lt;a class="reference external" href="https://en.wikipedia.org/wiki/CUDA#GPUs_supported"&gt;similar CUDA compute capabilities&lt;/a&gt;. For example for cost optimization reasons you may want to use A100s on your client and workers and a T4 on your scheduler.&lt;/p&gt;
&lt;p&gt;Users who do not have a GPU on the client and are leveraging GPU workers shouldn’t run into this as the GPU objects will only exist on the workers.&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/04/14/scheduler-environment-requirements.md&lt;/span&gt;, line 29)&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-doing-this"&gt;
&lt;h1&gt;Why are we doing this?&lt;/h1&gt;
&lt;p&gt;The reason we now suggest that you have the same hardware/software capabilities on the scheduler is that we are giving the scheduler the ability to deserialize graphs before distributing them to the workers. This will allow the scheduler to make smarter scheduling decisions in the future by having a better understanding of the operation it is performing.&lt;/p&gt;
&lt;p&gt;The downside to this is that graphs can contain complex Python objects created by any number of dependencies on the client side, so in order for the scheduler to deserialize them it needs to have the same libraries installed. Equally, if the client-side packages create GPU objects then the scheduler will also need one.&lt;/p&gt;
&lt;p&gt;We are sure you’ll agree that this breakage for a small percentage of users will be worth it for the long-term improvements to Dask.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/04/14/scheduler-environment-requirements/"/>
    <summary>Update May 3rd 2023: Clarify GPU recommendations.</summary>
    <category term="IO" label="IO"/>
    <category term="dataframe" label="dataframe"/>
    <published>2023-04-14T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2023/04/12/from-map/</id>
    <title>Deep Dive into creating a Dask DataFrame Collection with from_map</title>
    <updated>2023-04-12T00:00:00+00:00</updated>
    <author>
      <name>Rick Zamora</name>
    </author>
    <content type="html">&lt;p&gt;&lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe.html"&gt;Dask DataFrame&lt;/a&gt; provides dedicated IO functions for several popular tabular-data formats, like CSV and Parquet. If you are working with a supported format, then the corresponding function (e.g &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_csv&lt;/span&gt;&lt;/code&gt;) is likely to be the most reliable way to create a new Dask DataFrame collection. For other workflows, &lt;a class="reference external" href="https://docs.dask.org/en/stable/generated/dask.dataframe.from_map.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; now offers a convenient way to define a DataFrame collection as an arbitrary function mapping. While these kinds of workflows have historically required users to adopt the Dask Delayed API, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; now makes custom collection creation both easier and more performant.&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/04/12/from-map.md&lt;/span&gt;, line 11)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="what-is-from-map"&gt;

&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; API was added to Dask DataFrame in v2022.05.1 with the intention of replacing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_delayed&lt;/span&gt;&lt;/code&gt; as the recommended means of custom DataFrame creation. At its core, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; simply converts each element of an iterable object (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;inputs&lt;/span&gt;&lt;/code&gt;) into a distinct Dask DataFrame partition, using a common function (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;func&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;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;func&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Callable&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;iterable&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Iterable&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;dd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The overall behavior is essentially the Dask DataFrame equivalent of the standard-Python &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map&lt;/span&gt;&lt;/code&gt; function:&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="nb"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;func&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Callable&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;iterable&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Iterable&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;Iterator&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Note that both &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map&lt;/span&gt;&lt;/code&gt; actually support an arbitrary number of iterable inputs. However, we will only focus on the use of a single iterable argument 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/04/12/from-map.md&lt;/span&gt;, line 27)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="a-simple-example"&gt;
&lt;h1&gt;A simple example&lt;/h1&gt;
&lt;p&gt;To better understand the behavior of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt;, let’s consider the simple case that we want to interact with Feather-formatted data created with the following Pandas code:&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;pandas&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;pd&lt;/span&gt;

&lt;span class="n"&gt;size&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;
&lt;span class="n"&gt;paths&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;./data.0.feather&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;./data.1.feather&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;path&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;enumerate&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;paths&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;size&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;a&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;size&lt;/span&gt;
    &lt;span class="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;xyz&amp;quot;&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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;A&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;a&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;B&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;b&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;index&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;index&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;to_feather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Since Dask does not yet offer a dedicated &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_feather&lt;/span&gt;&lt;/code&gt; function (as of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-2023.3.1&lt;/span&gt;&lt;/code&gt;), most users would assume that the only option to create a Dask DataFrame collection is to use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.delayed&lt;/span&gt;&lt;/code&gt;. The “best practice” for creating a collection in this case, however, is to wrap &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pd.read_feather&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cudf.read_feather&lt;/span&gt;&lt;/code&gt; in a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; call like so:&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="gp"&gt;&amp;gt;&amp;gt;&amp;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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&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;from_map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_feather&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;paths&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                   A       B  index&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;               int64  object  int64&lt;/span&gt;
&lt;span class="go"&gt;                 ...     ...    ...&lt;/span&gt;
&lt;span class="go"&gt;                 ...     ...    ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: read_feather, 1 graph layer&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Which produces the following Pandas (or cuDF) object after computation:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;   A  B  index&lt;/span&gt;
&lt;span class="go"&gt;0  0  x      0&lt;/span&gt;
&lt;span class="go"&gt;1  0  y      1&lt;/span&gt;
&lt;span class="go"&gt;2  0  z      2&lt;/span&gt;
&lt;span class="go"&gt;0  1  x      3&lt;/span&gt;
&lt;span class="go"&gt;1  1  y      4&lt;/span&gt;
&lt;span class="go"&gt;2  1  z      5&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Although the same output can be achieved using the conventional &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dd.from_delayed&lt;/span&gt;&lt;/code&gt; strategy, using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; will improve the available opportunities for task-graph optimization within Dask.&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/04/12/from-map.md&lt;/span&gt;, line 74)&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="performance-considerations-specifying-meta-and-divisions"&gt;
&lt;h1&gt;Performance considerations: Specifying &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;divisions&lt;/span&gt;&lt;/code&gt;&lt;/h1&gt;
&lt;p&gt;Although &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;func&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;iterable&lt;/span&gt;&lt;/code&gt; are the only &lt;em&gt;required&lt;/em&gt; arguments to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt;, one can significantly improve the overall performance of a workflow by specifying optional arguments like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;divisions&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Due to the lazy nature of Dask DataFrame, each collection is required to carry around a schema (column name and dtype information) in the form of an empty Pandas (or cuDF) object. If &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; is not directly provided to the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; function, the schema will need to be populated by eagerly materializing the first partition, which can increase the apparent latency of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; API call itself. For this reason, it is always recommended to specify an explicit &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; argument if the expected column names and dtypes are known a priori.&lt;/p&gt;
&lt;p&gt;While passing in a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; argument is likely to reduce the&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; API call latency, passing in a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;divisions&lt;/span&gt;&lt;/code&gt; argument makes it possible to reduce the end-to-end compute time. This is because, by specifying &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;divisions&lt;/span&gt;&lt;/code&gt;, we are allowing Dask DataFrame to track useful per-partition min/max statistics. Therefore, if the overall workflow involves grouping or joining on the index, Dask can avoid the need to perform unnecessary shuffling operations.&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/04/12/from-map.md&lt;/span&gt;, line 82)&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="using-from-map-to-implement-a-custom-api"&gt;
&lt;h1&gt;Using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; to implement a custom API&lt;/h1&gt;
&lt;p&gt;Although it is currently difficult to automatically extract division information from the metadata of an arbitrary Feather dataset, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; makes it relatively easy to implement your own highly-functional &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_feather&lt;/span&gt;&lt;/code&gt; API using &lt;a class="reference external" href="https://arrow.apache.org/docs/python/index.html"&gt;PyArrow&lt;/a&gt;. For example, the following code is all that one needs to enable lazy Feather IO with both column projection and index selection:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;from_arrow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;table&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;(Optional) Utility to enforce &amp;#39;backend&amp;#39; configuration&amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;config&lt;/span&gt;

    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dataframe.backend&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;cudf&amp;quot;&lt;/span&gt;&lt;span class="p"&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;cudf&lt;/span&gt;

        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_arrow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;table&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="k"&gt;else&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;table&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_pandas&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;


&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;read_feather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;paths&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="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;Create a Dask DataFrame from Feather files&lt;/span&gt;

&lt;span class="sd"&gt;    Example of a &amp;quot;custom&amp;quot; `from_map` IO function&lt;/span&gt;

&lt;span class="sd"&gt;    Parameters&lt;/span&gt;
&lt;span class="sd"&gt;    ----------&lt;/span&gt;
&lt;span class="sd"&gt;    paths: list&lt;/span&gt;
&lt;span class="sd"&gt;        List of Feather-formatted paths. Each path will&lt;/span&gt;
&lt;span class="sd"&gt;        be mapped to a distinct DataFrame partition.&lt;/span&gt;
&lt;span class="sd"&gt;    columns: list or None, default None&lt;/span&gt;
&lt;span class="sd"&gt;        Optional list of columns to select from each file.&lt;/span&gt;
&lt;span class="sd"&gt;    index: str or None, default None&lt;/span&gt;
&lt;span class="sd"&gt;        Optional column name to set as the DataFrame index.&lt;/span&gt;

&lt;span class="sd"&gt;    Returns&lt;/span&gt;
&lt;span class="sd"&gt;    -------&lt;/span&gt;
&lt;span class="sd"&gt;    dask.dataframe.DataFrame&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;quot;&amp;quot;&amp;quot;&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pyarrow.dataset&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;ds&lt;/span&gt;

    &lt;span class="c1"&gt;# Step 1: Extract `meta` from the dataset&lt;/span&gt;
    &lt;span class="n"&gt;dataset&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ds&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;paths&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="nb"&gt;format&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;feather&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;meta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;from_arrow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dataset&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;schema&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;empty_table&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
    &lt;span class="n"&gt;meta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_index&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="k"&gt;else&lt;/span&gt; &lt;span class="n"&gt;meta&lt;/span&gt;
    &lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="ow"&gt;or&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;meta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

    &lt;span class="c1"&gt;# Step 2: Define the `func` argument&lt;/span&gt;
    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;func&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;frag&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="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="c1"&gt;# Create a Pandas DataFrame from a dataset fragment&lt;/span&gt;
        &lt;span class="c1"&gt;# NOTE: In practice, this function should&lt;/span&gt;
        &lt;span class="c1"&gt;# always be defined outside `read_feather`&lt;/span&gt;
        &lt;span class="k"&gt;assert&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="ow"&gt;is&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;
        &lt;span class="n"&gt;read_columns&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;
        &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="ow"&gt;and&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
            &lt;span class="n"&gt;read_columns&lt;/span&gt; &lt;span class="o"&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="n"&gt;index&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;from_arrow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;frag&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_table&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="n"&gt;read_columns&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_index&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="k"&gt;else&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;
        &lt;span class="k"&gt;return&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;columns&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="k"&gt;else&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;

    &lt;span class="c1"&gt;# Step 3: Define the `iterable` argument&lt;/span&gt;
    &lt;span class="n"&gt;iterable&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dataset&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_fragments&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

    &lt;span class="c1"&gt;# Step 4: Call `from_map`&lt;/span&gt;
    &lt;span class="k"&gt;return&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;from_map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
        &lt;span class="n"&gt;func&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="n"&gt;iterable&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="c1"&gt;# `func` kwarg&lt;/span&gt;
        &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;  &lt;span class="c1"&gt;# `func` kwarg&lt;/span&gt;
    &lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here we see that using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; to enable completely-lazy collection creation only requires four steps. First, we use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pyarrow.dataset&lt;/span&gt;&lt;/code&gt; to define a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; argument for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt;, so that we can avoid the unnecessary overhead of an eager read operation. For some file formats and/or applications, it may also be possible to calculate &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;divisions&lt;/span&gt;&lt;/code&gt; at this point. However, as explained above, such information is not readily available for this particular example.&lt;/p&gt;
&lt;p&gt;The second step is to define the underlying function (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;func&lt;/span&gt;&lt;/code&gt;) that we will use to produce each of our final DataFrame partitions. Third, we define one or more iterable objects containing the unique information needed to produce each partition (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;iterable&lt;/span&gt;&lt;/code&gt;). In this case, the only iterable object corresponds to a generator of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pyarrow.dataset&lt;/span&gt;&lt;/code&gt; fragments, which is essentially a wrapper around the input path list.&lt;/p&gt;
&lt;p&gt;The fourth and final step is to use the final &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;func&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;interable&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; information to call the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; API. Note that we also use this opportunity to specify additional key-word arguments, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;columns&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;index&lt;/span&gt;&lt;/code&gt;. In contrast to the iterable positional arguments, which are always mapped to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;func&lt;/span&gt;&lt;/code&gt;, these key-word arguments will be broadcasted.&lt;/p&gt;
&lt;p&gt;Using the&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_feather&lt;/span&gt;&lt;/code&gt; implementation above, it becomes both easy and efficient to convert an arbitrary Feather dataset into a lazy Dask DataFrame collection:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;read_feather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;paths&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;A&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;index&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                   A&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;               int64&lt;/span&gt;
&lt;span class="go"&gt;                 ...&lt;/span&gt;
&lt;span class="go"&gt;                 ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: func, 1 graph layer&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;       A&lt;/span&gt;
&lt;span class="go"&gt;index&lt;/span&gt;
&lt;span class="go"&gt;0      0&lt;/span&gt;
&lt;span class="go"&gt;1      0&lt;/span&gt;
&lt;span class="go"&gt;2      0&lt;/span&gt;
&lt;span class="go"&gt;3      1&lt;/span&gt;
&lt;span class="go"&gt;4      1&lt;/span&gt;
&lt;span class="go"&gt;5      1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/04/12/from-map.md&lt;/span&gt;, line 183)&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="advanced-enhancing-column-projection"&gt;
&lt;h1&gt;Advanced: Enhancing column projection&lt;/h1&gt;
&lt;p&gt;Although a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_feather&lt;/span&gt;&lt;/code&gt; implementation like the one above is likely to meet the basic needs of most applications, it is certainly possible that users will often leave out the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;column&lt;/span&gt;&lt;/code&gt; argument in practice. For example:&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;a&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;read_feather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;paths&lt;/span&gt;&lt;span class="p"&gt;)[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;A&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;For code like this, as the implementation currently stands, each IO task would be forced to read in an entire Feather file, and then select the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;”A”&lt;/span&gt;&lt;/code&gt; column from a Pandas/cuDF DataFrame only after it had already been read into memory. The additional overhead is insignificant for the toy-dataset used here. However, avoiding this kind of unnecessary IO can lead to dramatic performance improvements in real-world applications.&lt;/p&gt;
&lt;p&gt;So, how can we modify our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_feather&lt;/span&gt;&lt;/code&gt; implementation to take advantage of external column-projection operations (like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ddf[&amp;quot;A&amp;quot;]&lt;/span&gt;&lt;/code&gt;)? The good news is that &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; is already equipped with the necessary graph-optimization hooks to handle this, so long as the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;func&lt;/span&gt;&lt;/code&gt; object satisfies the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DataFrameIOFunction&lt;/span&gt;&lt;/code&gt; protocol:&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="nd"&gt;@runtime_checkable&lt;/span&gt;
&lt;span class="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;DataFrameIOFunction&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Protocol&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;DataFrame IO function with projectable columns&lt;/span&gt;
&lt;span class="sd"&gt;    Enables column projection in ``DataFrameIOLayer``.&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;

    &lt;span class="nd"&gt;@property&lt;/span&gt;
    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;Return the current column projection&amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
        &lt;span class="k"&gt;raise&lt;/span&gt; &lt;span class="ne"&gt;NotImplementedError&lt;/span&gt;

    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;project_columns&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;Return a new DataFrameIOFunction object&lt;/span&gt;
&lt;span class="sd"&gt;        with a new column projection&lt;/span&gt;
&lt;span class="sd"&gt;        &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
        &lt;span class="k"&gt;raise&lt;/span&gt; &lt;span class="ne"&gt;NotImplementedError&lt;/span&gt;

    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="fm"&gt;__call__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;args&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;Return a new DataFrame partition&amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
        &lt;span class="k"&gt;raise&lt;/span&gt; &lt;span class="ne"&gt;NotImplementedError&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;That is, all we need to do is change “Step 2” of our implementation to use the following code instead:&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;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe.io.utils&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;DataFrameIOFunction&lt;/span&gt;

    &lt;span class="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;ReadFeather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;DataFrameIOFunction&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;        &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;Create a Pandas/cuDF DataFrame from a dataset fragment&amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
        &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="fm"&gt;__init__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
            &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_columns&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;
            &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;

        &lt;span class="nd"&gt;@property&lt;/span&gt;
        &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
            &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_columns&lt;/span&gt;

        &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;project_columns&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
            &lt;span class="c1"&gt;# Replace this object with one that will only read `columns`&lt;/span&gt;
            &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="o"&gt;!=&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;ReadFeather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;

        &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="fm"&gt;__call__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;frag&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
            &lt;span class="c1"&gt;# Same logic as original `func`&lt;/span&gt;
            &lt;span class="n"&gt;read_columns&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;
            &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="ow"&gt;and&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                &lt;span class="n"&gt;read_columns&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&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="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&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;from_arrow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;frag&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_table&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="n"&gt;read_columns&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;df&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set_index&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;index&lt;/span&gt; &lt;span class="k"&gt;else&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;
            &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="k"&gt;else&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;

    &lt;span class="n"&gt;func&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ReadFeather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/04/12/from-map.md&lt;/span&gt;, line 251)&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;It is now easier than ever to create a Dask DataFrame collection from an arbitrary data source. Although the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.delayed&lt;/span&gt;&lt;/code&gt; API has already enabled similar functionality for many years, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; now makes it possible to implement a custom IO function without sacrificing any of the high-level graph optimizations leveraged by the rest of the Dask DataFrame API.&lt;/p&gt;
&lt;p&gt;Start experimenting with &lt;a class="reference external" href="https://docs.dask.org/en/stable/generated/dask.dataframe.from_map.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; today, and let us know how it goes!&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/04/12/from-map/"/>
    <summary>Dask DataFrame provides dedicated IO functions for several popular tabular-data formats, like CSV and Parquet. If you are working with a supported format, then the corresponding function (e.g read_csv) is likely to be the most reliable way to create a new Dask DataFrame collection. For other workflows, from_map now offers a convenient way to define a DataFrame collection as an arbitrary function mapping. While these kinds of workflows have historically required users to adopt the Dask Delayed API, from_map now makes custom collection creation both easier and more performant.</summary>
    <category term="IO" label="IO"/>
    <category term="dataframe" label="dataframe"/>
    <published>2023-04-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2022/08/09/understanding-meta-keyword-argument/</id>
    <title>Understanding Dask’s meta keyword argument</title>
    <updated>2022-08-09T00:00:00+00:00</updated>
    <author>
      <name>Pavithra Eswaramoorthy</name>
    </author>
    <content type="html">&lt;p&gt;If you have worked with Dask DataFrames or Dask Arrays, you have probably come across the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; keyword argument. Perhaps, while using methods like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apply()&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pandas&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;pd&lt;/span&gt;

&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&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;datasets&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;timeseries&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;


&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;my_custom_arithmetic&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;r&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;r&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;id&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;r&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="n"&gt;r&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;r&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;r&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;
    &lt;span class="k"&gt;else&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;


&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;my_computation&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;apply&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
    &lt;span class="n"&gt;my_custom_arithmetic&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Series&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;float64&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;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;head&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="c1"&gt;# Output:&lt;/span&gt;
&lt;span class="c1"&gt;#&lt;/span&gt;
&lt;span class="c1"&gt;#                        id    name         x         y  my_computation&lt;/span&gt;
&lt;span class="c1"&gt;# timestamp&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:00  1055  Victor -0.575374  0.868320        2.067696&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:01   994   Zelda  0.963684  0.972240        0.000000&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:02   982  George -0.997531 -0.876222        0.000000&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:03   981  Ingrid  0.852159 -0.419733        0.000000&lt;/span&gt;
&lt;span class="c1"&gt;# 2000-01-01 00:00:04  1029   Jerry -0.839431 -0.736572       -0.768500&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You might have also seen one or more of the following warnings/errors:&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="ne"&gt;UserWarning&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;You&lt;/span&gt; &lt;span class="n"&gt;did&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="n"&gt;provide&lt;/span&gt; &lt;span class="n"&gt;metadata&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;so&lt;/span&gt; &lt;span class="n"&gt;Dask&lt;/span&gt; &lt;span class="ow"&gt;is&lt;/span&gt; &lt;span class="n"&gt;running&lt;/span&gt; &lt;span class="n"&gt;your&lt;/span&gt; &lt;span class="n"&gt;function&lt;/span&gt; &lt;span class="n"&gt;on&lt;/span&gt; &lt;span class="n"&gt;a&lt;/span&gt; &lt;span class="n"&gt;small&lt;/span&gt; &lt;span class="n"&gt;dataset&lt;/span&gt; &lt;span class="n"&gt;to&lt;/span&gt; &lt;span class="n"&gt;guess&lt;/span&gt; &lt;span class="n"&gt;output&lt;/span&gt; &lt;span class="n"&gt;types&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt; &lt;span class="n"&gt;It&lt;/span&gt; &lt;span class="ow"&gt;is&lt;/span&gt; &lt;span class="n"&gt;possible&lt;/span&gt; &lt;span class="n"&gt;that&lt;/span&gt; &lt;span class="n"&gt;Dask&lt;/span&gt; &lt;span class="n"&gt;will&lt;/span&gt; &lt;span class="n"&gt;guess&lt;/span&gt; &lt;span class="n"&gt;incorrectly&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;UserWarning: `meta` is not specified, inferred from partial data. Please provide `meta` if the result is unexpected.
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;ValueError: Metadata inference failed in …
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If the above messages look familiar, this blog post is for you. :)&lt;/p&gt;
&lt;p&gt;We will discuss:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;what the is &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; keyword argument,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;why does Dask need &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;, and&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;how to use it effectively.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We will look at &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; mainly in the context of Dask DataFrames, however, similar principles also apply to Dask Arrays.&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/2022/08/09/understanding-meta-keyword-argument.md&lt;/span&gt;, line 65)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="what-is-meta"&gt;

&lt;p&gt;Before answering this, let’s quickly discuss &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe.html"&gt;Dask DataFrames&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;A Dask DataFrame is a lazy object composed of multiple &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html"&gt;pandas DataFrames&lt;/a&gt;, where each pandas DataFrame is called a “partition”. These are stacked along the index and Dask keeps track of these partitions using “divisions”, which is a tuple representing the start and end index of each partition.&lt;/p&gt;
&lt;img src="https://docs.dask.org/en/stable/_images/dask-dataframe.svg" alt="Dask DataFrame consists of multiple pandas DataFrames" width="50%"&gt;
&lt;p&gt;When you create a Dask DataFrame, you usually see something like the following:&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;pandas&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;pd&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&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;DataFrame&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="p"&gt;{&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;        &lt;span class="s2"&gt;&amp;quot;x&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;6&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;        &lt;span class="s2"&gt;&amp;quot;y&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;16&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="p"&gt;},&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;npartitions&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                   x      y&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0              int64  int64&lt;/span&gt;
&lt;span class="go"&gt;3                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;5                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: from_pandas, 2 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here, Dask has created the structure of the DataFrame using some “metadata” information about the &lt;em&gt;column names&lt;/em&gt; and their &lt;em&gt;datatypes&lt;/em&gt;. This metadata information is called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;. Dask uses &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; for understanding Dask operations and creating accurate task graphs (i.e., the logic of your computation).&lt;/p&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; &lt;em&gt;keyword argument&lt;/em&gt; in various Dask DataFrame functions allows you to explicitly share this metadata information with Dask. Note that the keyword argument is concerned with the metadata of the &lt;em&gt;output&lt;/em&gt; of those functions.&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/2022/08/09/understanding-meta-keyword-argument.md&lt;/span&gt;, line 99)&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-does-dask-need-meta"&gt;
&lt;h1&gt;Why does Dask need &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;?&lt;/h1&gt;
&lt;p&gt;Dask computations are evaluated &lt;em&gt;lazily&lt;/em&gt;. This means Dask creates the logic and flow, called task graph, of the computation immediately, but evaluates them only when necessary – usually, on calling &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.compute()&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;An example task graph generated to compute the sum of the DataFrame:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;
&lt;span class="go"&gt;Dask Series Structure:&lt;/span&gt;
&lt;span class="go"&gt;npartitions=1&lt;/span&gt;
&lt;span class="go"&gt;x    int64&lt;/span&gt;
&lt;span class="go"&gt;y      ...&lt;/span&gt;
&lt;span class="go"&gt;dtype: int64&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: dataframe-sum-agg, 5 tasks&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;img src="/images/understanding-meta-task-graph.png" alt="Dask task graph, starts with two partitions thatare input to a dataframe-sum-chunk task each. Their results are input to a single dataframe-sum-agg which produces the final output." width="50%"&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;x    15&lt;/span&gt;
&lt;span class="go"&gt;y    75&lt;/span&gt;
&lt;span class="go"&gt;dtype: int64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is a single operation, but Dask workflows usually have multiple such operation chained together. Therefore, to create the task graph effectively, Dask needs to know the strucutre and datatypes of the DataFame after each operation. Especially because Dask does not know the actual values/structure of the DataFrame yet.&lt;/p&gt;
&lt;p&gt;This is where &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; is comes in.&lt;/p&gt;
&lt;p&gt;In the above example, the Dask DataFrame changed into a Series after &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum()&lt;/span&gt;&lt;/code&gt;. Dask knows this (even before we call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;compute()&lt;/span&gt;&lt;/code&gt;) only becasue of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Internally, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; is represented as an empty pandas &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe.html"&gt;DataFrame&lt;/a&gt; or &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.Series.html"&gt;Series&lt;/a&gt;, which has the same structure as the Dask DataFrame. To learn more about how &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; is defined internally, check out the &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe-design.html#metadata"&gt;DataFrame Internal Design documentation&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;To see the actual metadata information for a collection, you can look at the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt; attribute[1]:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_meta&lt;/span&gt;
&lt;span class="go"&gt;Series([], dtype: int64)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/2022/08/09/understanding-meta-keyword-argument.md&lt;/span&gt;, line 142)&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="how-to-specify-meta"&gt;
&lt;h1&gt;How to specify &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;?&lt;/h1&gt;
&lt;p&gt;You can specify &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; in a few different ways, but the recommended way for Dask DataFrame is:&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;“An empty &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pd.DataFrame&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pd.Series&lt;/span&gt;&lt;/code&gt; that matches the dtypes and column names of the output.”&lt;/p&gt;
&lt;p&gt;~ &lt;a class="reference external" href="https://docs.dask.org/en/stable/generated/dask.dataframe.DataFrame.apply.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DataFrame.apply()&lt;/span&gt;&lt;/code&gt; docstring&lt;/a&gt;&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;For example:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;meta_df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&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;x&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;y&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;meta_df&lt;/span&gt;

&lt;span class="go"&gt;Empty DataFrame&lt;/span&gt;
&lt;span class="go"&gt;Columns: [x, y]&lt;/span&gt;
&lt;span class="go"&gt;Index: []&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;apply&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;meta_df&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf2&lt;/span&gt;
&lt;span class="go"&gt;   x  y&lt;/span&gt;
&lt;span class="go"&gt;0  0  0&lt;/span&gt;
&lt;span class="go"&gt;1  1  1&lt;/span&gt;
&lt;span class="go"&gt;2  2  2&lt;/span&gt;
&lt;span class="go"&gt;3  0  3&lt;/span&gt;
&lt;span class="go"&gt;4  1  4&lt;/span&gt;
&lt;span class="go"&gt;5  2  5&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The &lt;a class="reference external" href="https://docs.dask.org/en/stable/dataframe-design.html#metadata"&gt;other ways you can describe &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; are:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;For a DataFrame, you can specify &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; as a:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Python dictionary: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;{column_name_1:&lt;/span&gt; &lt;span class="pre"&gt;dtype_1,&lt;/span&gt; &lt;span class="pre"&gt;column_name_2:&lt;/span&gt; &lt;span class="pre"&gt;dtype_2,&lt;/span&gt; &lt;span class="pre"&gt;…}&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Iterable of tuples: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;[(column_name_1,&lt;/span&gt; &lt;span class="pre"&gt;dtype_1),&lt;/span&gt; &lt;span class="pre"&gt;(columns_name_2,&lt;/span&gt; &lt;span class="pre"&gt;dtype_2,&lt;/span&gt; &lt;span class="pre"&gt;…)]&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;&lt;strong&gt;Note&lt;/strong&gt; that while describing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; as shown above, using a dictionary or iterable of tuples, the order in which you mention column names is important. Dask will use the same order to create the pandas DataFrame for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;. If the orders don’t match, you will see the following error:&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="ne"&gt;ValueError&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;The&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;the&lt;/span&gt; &lt;span class="n"&gt;computed&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="n"&gt;do&lt;/span&gt; &lt;span class="ow"&gt;not&lt;/span&gt; &lt;span class="n"&gt;match&lt;/span&gt; &lt;span class="n"&gt;the&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;the&lt;/span&gt; &lt;span class="n"&gt;provided&lt;/span&gt; &lt;span class="n"&gt;metadata&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For a Series output, you can specify &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; using a single tuple: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(coulmn_name,&lt;/span&gt; &lt;span class="pre"&gt;dtype)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;You should &lt;strong&gt;not&lt;/strong&gt; describe &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; using just a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dtype&lt;/span&gt;&lt;/code&gt; (like: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta=&amp;quot;int64&amp;quot;&lt;/span&gt;&lt;/code&gt;), even for scalar outputs. If you do, you will see the following warning:&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;FutureWarning: Meta is not valid, `map_partitions` and `map_overlap` expects output to be a pandas object. Try passing a pandas object as meta or a dict or tuple representing the (name, dtype) of the columns. In the future the meta you passed will not work.
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;During operations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_partitions&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apply&lt;/span&gt;&lt;/code&gt; (which uses &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_partitions&lt;/span&gt;&lt;/code&gt; internally), Dask coerces the scalar output of each partition into a pandas object. So, the output of functions that take &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; will never be scalar.&lt;/p&gt;
&lt;p&gt;For example:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;repartition&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;npartitions&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="nb"&gt;type&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;pandas.core.series.Series&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here, the Dask DataFrame &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ddf&lt;/span&gt;&lt;/code&gt; has only one partition. Hence, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;len(x)&lt;/span&gt;&lt;/code&gt; on that one partition would result in a scalar output of integer dtype. However, when we compute it, we see a pandas Series. This confirms that Dask is coercing the outputs to pandas objects.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Another note&lt;/strong&gt;, Dask Arrays may not always do this conversion. You can look at the &lt;a class="reference external" href="https://docs.dask.org/en/stable/array-api.html"&gt;API reference&lt;/a&gt; for your particular Array operation for details.&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;numpy&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;np&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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.array&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;da&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;my_arr&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;my_arr&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_blocks&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;len&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;10&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/2022/08/09/understanding-meta-keyword-argument.md&lt;/span&gt;, line 215)&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="meta-does-not-force-the-structure-or-dtypes"&gt;
&lt;h1&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; does not &lt;em&gt;force&lt;/em&gt; the structure or dtypes&lt;/h1&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; can be thought of as a suggestion to Dask. Dask uses this &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; to generate the task graph until it can infer the actual metadata from the values. It &lt;strong&gt;does not&lt;/strong&gt; force the output to have the structure or dtype of the specified &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Consider the following example, and remember that we defined &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ddf&lt;/span&gt;&lt;/code&gt; with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;y&lt;/span&gt;&lt;/code&gt; column names in the previous sections.&lt;/p&gt;
&lt;p&gt;If we provide different column names (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;b&lt;/span&gt;&lt;/code&gt;) in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; description, Dask uses these new names to create the task graph:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;meta_df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&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;a&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;b&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;apply&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;meta&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;meta_df&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                   a      b&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0              int64  int64&lt;/span&gt;
&lt;span class="go"&gt;3                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;5                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: apply, 4 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;However, if we compute &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;result&lt;/span&gt;&lt;/code&gt;, we will get the following error:&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;ValueError: The columns in the computed data do not match the columns in the provided metadata&lt;/span&gt;
&lt;span class="go"&gt;  Extra:   [&amp;#39;x&amp;#39;, &amp;#39;y&amp;#39;]&lt;/span&gt;
&lt;span class="go"&gt;  Missing: [&amp;#39;a&amp;#39;, &amp;#39;b&amp;#39;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;While computing, Dask evaluates the actual metadata with columns &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;y&lt;/span&gt;&lt;/code&gt;. This does not match the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; that we provided, and hence, Dask raises a helpful error message. Notice how Dask does not change the output to have &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;b&lt;/span&gt;&lt;/code&gt; here, rather uses &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;b&lt;/span&gt;&lt;/code&gt; column names only for intermediate task graphs.&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/2022/08/09/understanding-meta-keyword-argument.md&lt;/span&gt;, line 247)&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="using-meta-directly"&gt;
&lt;h1&gt;Using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt; directly&lt;/h1&gt;
&lt;p&gt;In some rare case, you can also set the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt; attribute[1] directly for a Dask DataFrame. For example, if the DataFrame was created with incorrect dtypes, like:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&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;DataFrame&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="p"&gt;{&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;        &lt;span class="s2"&gt;&amp;quot;x&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;6&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;        &lt;span class="s2"&gt;&amp;quot;y&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;16&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="p"&gt;},&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;object&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="c1"&gt;# Note the “object” dtype&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;npartitions&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                    x       y&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0              object  object&lt;/span&gt;
&lt;span class="go"&gt;3                 ...     ...&lt;/span&gt;
&lt;span class="go"&gt;5                 ...     ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: from_pandas, 2 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The values are clearly integers but the dtype says &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;object&lt;/span&gt;&lt;/code&gt;, so we can’t perform integer operations like addition:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="go"&gt;ValueError: Metadata inference failed in `add`.&lt;/span&gt;

&lt;span class="go"&gt;Original error is below:&lt;/span&gt;
&lt;span class="go"&gt;------------------------&lt;/span&gt;
&lt;span class="go"&gt;TypeError(&amp;#39;can only concatenate str (not &amp;quot;int&amp;quot;) to str&amp;#39;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here, we can explicitly define &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt;[1]:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_meta&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&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;x&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;y&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;int64&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;Then, perform the addition:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ddf&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                   x      y&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0              int64  int64&lt;/span&gt;
&lt;span class="go"&gt;3                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;5                ...    ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: add, 4 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Thanks for reading!&lt;/p&gt;
&lt;p&gt;Have you run into issues with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;meta&lt;/span&gt;&lt;/code&gt; before? Please let us know on &lt;a class="reference external" href="https://dask.discourse.group/"&gt;Discourse&lt;/a&gt;, and we will consider including it here, or updating the Dask documentation. :)&lt;/p&gt;
&lt;p&gt;[1] NOTE: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt; is not a public property, so we recommend using it only when necessary. There is &lt;a class="reference external" href="https://github.com/dask/dask/issues/8585"&gt;an ongoing discussion&lt;/a&gt; around creating public methods to get, set, and view the information in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;._meta&lt;/span&gt;&lt;/code&gt;, and this blog post will be updated to use the public methods when they’re created.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2022/08/09/understanding-meta-keyword-argument/"/>
    <summary>If you have worked with Dask DataFrames or Dask Arrays, you have probably come across the meta keyword argument. Perhaps, while using methods like apply():</summary>
    <category term="dataframe" label="dataframe"/>
    <published>2022-08-09T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/10/08/df-groupby/</id>
    <title>DataFrame Groupby Aggregations</title>
    <updated>2019-10-08T00:00:00+00:00</updated>
    <author>
      <name>Benjamin Zaitlen &amp; James Bourbeau</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/2019/10/08/df-groupby.md&lt;/span&gt;, line 10)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="groupby-aggregations-with-dask"&gt;

&lt;p&gt;In this post we’ll dive into how Dask computes groupby aggregations. These are commonly used operations for ETL and analysis in which we split data into groups, apply a function to each group independently, and then combine the results back together. In the PyData/R world this is often referred to as the split-apply-combine strategy (first coined by &lt;a class="reference external" href="https://www.jstatsoft.org/article/view/v040i01"&gt;Hadley Wickham&lt;/a&gt;) and is used widely throughout the &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/user_guide/groupby.html"&gt;Pandas ecosystem&lt;/a&gt;.&lt;/p&gt;
&lt;div align="center"&gt;
  &lt;a href="/images/split-apply-combine.png"&gt;
    &lt;img src="/images/split-apply-combine.png" width="80%" align="center"&gt;
  &lt;/a&gt;
  &lt;p align="center"&gt;&lt;i&gt;Image courtesy of swcarpentry.github.io&lt;/i&gt;&lt;/p&gt;
&lt;/div&gt;
&lt;p&gt;Dask leverages this idea using a similarly catchy name: apply-concat-apply or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; for short. Here we’ll explore the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; strategy in both simple and complex operations.&lt;/p&gt;
&lt;p&gt;First, recall that a Dask DataFrame is a &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-design.html#internal-design"&gt;collection&lt;/a&gt; of DataFrame objects (e.g. each &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-design.html#partitions"&gt;partition&lt;/a&gt; of a Dask DataFrame is a Pandas DataFrame). For example, let’s say we have the following Pandas DataFrame:&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;pandas&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;pd&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;a&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;99&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;                       &lt;span class="n"&gt;b&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;12&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;9&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;                       &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;9&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;44&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;33&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;]))&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df&lt;/span&gt;
&lt;span class="go"&gt;     a   b   c&lt;/span&gt;
&lt;span class="go"&gt;0    1   1   2&lt;/span&gt;
&lt;span class="go"&gt;1    1   3   4&lt;/span&gt;
&lt;span class="go"&gt;2    2  10   5&lt;/span&gt;
&lt;span class="go"&gt;3    3   3   2&lt;/span&gt;
&lt;span class="go"&gt;4    3   2   3&lt;/span&gt;
&lt;span class="go"&gt;5    1   1   5&lt;/span&gt;
&lt;span class="go"&gt;6    1   3   2&lt;/span&gt;
&lt;span class="go"&gt;7    2  10   3&lt;/span&gt;
&lt;span class="go"&gt;8    3   3   9&lt;/span&gt;
&lt;span class="go"&gt;9    3   3   2&lt;/span&gt;
&lt;span class="go"&gt;10  99  12  44&lt;/span&gt;
&lt;span class="go"&gt;11  10   0  33&lt;/span&gt;
&lt;span class="go"&gt;12   1   9   2&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;To create a Dask DataFrame with three partitions from this data, we could partition &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df&lt;/span&gt;&lt;/code&gt; between the indices of: (0, 4), (5, 9), and (10, 12). We can perform this partitioning with Dask by using the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_pandas&lt;/span&gt;&lt;/code&gt; function with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;npartitions=3&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="gp"&gt;&amp;gt;&amp;gt;&amp;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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&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;from_pandas&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;npartitions&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The 3 partitions are simply 3 individual Pandas DataFrames:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;partitions&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;   a   b  c&lt;/span&gt;
&lt;span class="go"&gt;0  1   1  2&lt;/span&gt;
&lt;span class="go"&gt;1  1   3  4&lt;/span&gt;
&lt;span class="go"&gt;2  2  10  5&lt;/span&gt;
&lt;span class="go"&gt;3  3   3  2&lt;/span&gt;
&lt;span class="go"&gt;4  3   2  3&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/2019/10/08/df-groupby.md&lt;/span&gt;, line 66)&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="apply-concat-apply"&gt;
&lt;h1&gt;Apply-concat-apply&lt;/h1&gt;
&lt;p&gt;When Dask applies a function and/or algorithm (e.g. &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt;, etc.) to a Dask DataFrame, it does so by applying that operation to all the constituent partitions independently, collecting (or concatenating) the outputs into intermediary results, and then applying the operation again to the intermediary results to produce a final result. Internally, Dask re-uses the same apply-concat-apply methodology for many of its internal DataFrame calculations.&lt;/p&gt;
&lt;p&gt;Let’s break down how Dask computes &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ddf.groupby(['a',&lt;/span&gt; &lt;span class="pre"&gt;'b']).c.sum()&lt;/span&gt;&lt;/code&gt; by going through each step in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; process. We’ll begin by splitting our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df&lt;/span&gt;&lt;/code&gt; Pandas DataFrame into three partitions:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df_1&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[:&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df_2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;df_3&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;:]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;section id="apply"&gt;
&lt;h2&gt;Apply&lt;/h2&gt;
&lt;p&gt;Next we perform the same &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby(['a',&lt;/span&gt; &lt;span class="pre"&gt;'b']).c.sum()&lt;/span&gt;&lt;/code&gt; operation on each of our three partitions:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;sr1&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df_1&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;a&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;b&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;c&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;sr2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df_2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;a&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;b&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;c&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;sr3&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;df_3&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;a&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;b&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;c&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;These operations each produce a Series with a &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/user_guide/advanced.html"&gt;MultiIndex&lt;/a&gt;:&lt;/p&gt;
&lt;table&gt;
  &lt;tr&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; sr1
a  b
1  1     2
   3     4
2  10    5
3  2     3
   3     2
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; sr2
a  b
1  1      5
   3      2
2  10     3
3  3     11
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; sr3
a   b
1   9      2
10  0     33
99  12    44
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
  &lt;/tr&gt;
&lt;/table&gt;
&lt;/section&gt;
&lt;section id="the-concat"&gt;
&lt;h2&gt;The conCat!&lt;/h2&gt;
&lt;p&gt;After the first &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apply&lt;/span&gt;&lt;/code&gt;, the next step is to concatenate the intermediate &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sr1&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sr2&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sr3&lt;/span&gt;&lt;/code&gt; results. This is fairly straightforward to do using the Pandas &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;concat&lt;/span&gt;&lt;/code&gt; function:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;sr_concat&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;concat&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;sr1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sr2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sr3&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;sr_concat&lt;/span&gt;
&lt;span class="go"&gt;a   b&lt;/span&gt;
&lt;span class="go"&gt;1   1      2&lt;/span&gt;
&lt;span class="go"&gt;    3      4&lt;/span&gt;
&lt;span class="go"&gt;2   10     5&lt;/span&gt;
&lt;span class="go"&gt;3   2      3&lt;/span&gt;
&lt;span class="go"&gt;    3      2&lt;/span&gt;
&lt;span class="go"&gt;1   1      5&lt;/span&gt;
&lt;span class="go"&gt;    3      2&lt;/span&gt;
&lt;span class="go"&gt;2   10     3&lt;/span&gt;
&lt;span class="go"&gt;3   3     11&lt;/span&gt;
&lt;span class="go"&gt;1   9      2&lt;/span&gt;
&lt;span class="go"&gt;10  0     33&lt;/span&gt;
&lt;span class="go"&gt;99  12    44&lt;/span&gt;
&lt;span class="go"&gt;Name: c, dtype: int64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="apply-redux"&gt;
&lt;h2&gt;Apply Redux&lt;/h2&gt;
&lt;p&gt;Our final step is to apply the same &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby(['a',&lt;/span&gt; &lt;span class="pre"&gt;'b']).c.sum()&lt;/span&gt;&lt;/code&gt; operation again on the concatenated &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sr_concat&lt;/span&gt;&lt;/code&gt; Series. However we no longer have columns &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;a&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;b&lt;/span&gt;&lt;/code&gt;, so how should we proceed?&lt;/p&gt;
&lt;p&gt;Zooming out a bit, our goal is to add the values in the column which have the same index. For example, there are two rows with the index &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;(1,&lt;/span&gt; &lt;span class="pre"&gt;1)&lt;/span&gt;&lt;/code&gt; with corresponding values: 2, 5. So how can we groupby the indices with the same value? A MutliIndex uses &lt;a class="reference external" href="https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.MultiIndex.html#pandas.MultiIndex"&gt;levels&lt;/a&gt; to define what the value is at a give index. Dask &lt;a class="reference external" href="https://github.com/dask/dask/blob/973c6e1b2e38c2d9d6e8c75fb9b4ab7a0d07e6a7/dask/dataframe/groupby.py#L69-L75"&gt;determines&lt;/a&gt; and &lt;a class="reference external" href="https://github.com/dask/dask/blob/973c6e1b2e38c2d9d6e8c75fb9b4ab7a0d07e6a7/dask/dataframe/groupby.py#L1065"&gt;uses these levels&lt;/a&gt; in the final apply step of the apply-concat-apply calculation. In our case, the level is &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;[0,&lt;/span&gt; &lt;span class="pre"&gt;1]&lt;/span&gt;&lt;/code&gt;, that is, we want both the index at the 0th level and the 1st level and if we group by both, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;0,&lt;/span&gt; &lt;span class="pre"&gt;1&lt;/span&gt;&lt;/code&gt;, we will have effectively grouped the same indices together:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;total&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sr_concat&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;level&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
  &lt;tr&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; total
a   b
1   1      7
    3      6
    9      2
2   10     8
3   2      3
    3     13
10  0     33
99  12    44
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; ddf.groupby(['a', 'b']).c.sum().compute()
a   b
1   1      7
    3      6
2   10     8
3   2      3
    3     13
1   9      2
10  0     33
99  12    44
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
    &lt;th&gt;
      &lt;pre&gt;
&gt;&gt;&gt; df.groupby(['a', 'b']).c.sum()
a   b
1   1      7
    3      6
    9      2
2   10     8
3   2      3
    3     13
10  0     33
99  12    44
Name: c, dtype: int64
      &lt;/pre&gt;
    &lt;/th&gt;
  &lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;Additionally, we can easily examine the steps of this apply-concat-apply calculation by &lt;a class="reference external" href="https://docs.dask.org/en/latest/graphviz.html"&gt;visualizing the task graph&lt;/a&gt; for the computation:&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;a&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;b&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;c&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/sum.svg"&gt;
  &lt;img src="/images/sum.svg" width="80%"&gt;
&lt;/a&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum&lt;/span&gt;&lt;/code&gt; is rather a straight-forward calculation. What about something a bit more complex like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupby&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;a&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;b&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;visualize&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/mean.svg"&gt;
  &lt;img src="/images/mean.svg" width="80%"&gt;
&lt;/a&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Mean&lt;/span&gt;&lt;/code&gt; is a good example of an operation which doesn’t directly fit in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; model – concatenating &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt; values and taking the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt; again will yield incorrect results. Like any style of computation: vectorization, Map/Reduce, etc., we sometime need to creatively fit the computation to the style/mode. In the case of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;aca&lt;/span&gt;&lt;/code&gt; we can often break down the calculation into constituent parts. For &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt;, this would be &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sum&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;count&lt;/span&gt;&lt;/code&gt;:&lt;/p&gt;
&lt;div class="math notranslate nohighlight"&gt;
\[ \bar{x} = \frac{x_1+x_2+\cdots +x_n}{n}\]&lt;/div&gt;
&lt;p&gt;From the task graph above, we can see that two independent tasks for each partition: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;series-groupby-count-chunk&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;series-groupby-sum-chunk&lt;/span&gt;&lt;/code&gt;. The results are then aggregated into two final nodes: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;series-groupby-count-agg&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;series-groupby-sum-agg&lt;/span&gt;&lt;/code&gt; and then we finally calculate the mean: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;total&lt;/span&gt; &lt;span class="pre"&gt;sum&lt;/span&gt; &lt;span class="pre"&gt;/&lt;/span&gt; &lt;span class="pre"&gt;total&lt;/span&gt; &lt;span class="pre"&gt;count&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/10/08/df-groupby/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="dask" label="dask"/>
    <category term="dataframe" label="dataframe"/>
    <published>2019-10-08T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/03/04/building-gpu-groupbys/</id>
    <title>Building GPU Groupby-Aggregations for Dask</title>
    <updated>2019-03-04T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</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/2019/03/04/building-gpu-groupbys.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;We’ve sufficiently aligned Dask DataFrame and cuDF to get groupby aggregations
like the following to work 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;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This post describes the kind of work we had to do as a model for future
development.&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/2019/03/04/building-gpu-groupbys.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&gt;
&lt;section id="plan"&gt;
&lt;h1&gt;Plan&lt;/h1&gt;
&lt;p&gt;As outlined in a previous post, &lt;a class="reference internal" href="#../../../2019/01/13/dask-cudf-first-steps.html"&gt;&lt;span class="xref myst"&gt;Dask, Pandas, and GPUs: first
steps&lt;/span&gt;&lt;/a&gt;, our plan to produce
distributed GPU dataframes was to combine &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe.html"&gt;Dask
DataFrame&lt;/a&gt; with
&lt;a class="reference external" href="https://rapids.ai"&gt;cudf&lt;/a&gt;. In particular, we had to&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;change Dask DataFrame so that it would parallelize not just around the
Pandas DataFrames that it works with today, but around anything that looked
enough like a Pandas DataFrame&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;change cuDF so that it would look enough like a Pandas DataFrame to fit
within the algorithms in Dask DataFrame&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/2019/03/04/building-gpu-groupbys.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="changes"&gt;
&lt;h1&gt;Changes&lt;/h1&gt;
&lt;p&gt;On the Dask side this mostly meant replacing&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Replacing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;isinstance(df,&lt;/span&gt; &lt;span class="pre"&gt;pd.DataFrame)&lt;/span&gt;&lt;/code&gt; checks with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_dataframe_like(df)&lt;/span&gt;&lt;/code&gt;
checks (after defining a suitable
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_dataframe_like&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_series_like&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;is_index_like&lt;/span&gt;&lt;/code&gt; functions&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Avoiding some more exotic functionality in Pandas, and instead trying to
use more common functionality that we can expect to be in most DataFrame
implementations&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;On the cuDF side this means making dozens of tiny changes to align the cuDF API
to the Pandas API, and to add in missing features.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Dask Changes:&lt;/strong&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4359"&gt;Remove explicit pandas checks and provide cudf lazy registration #4359&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4375"&gt;Replace isinstance(…, pandas) with is_dataframe_like #4375&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4395"&gt;Add has_parallel_type&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4396"&gt;Lazily register more cudf functions and move to backends file #4396&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4418"&gt;Avoid checking against types in is_dataframe_like #4418&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4470"&gt;Replace cudf-specific code with dask-cudf import #4470&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/dask/dask/pull/4482"&gt;Avoid groupby.agg(callable) in groupby-var #4482&lt;/a&gt; – this one is notable in that by simplifying our Pandas usage we actually got a significant speedup on the Pandas side.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;cuDF Changes:&lt;/strong&gt;&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/529"&gt;Build DataFrames from CUDA array libraries #529&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/534"&gt;Groupby AttributeError&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/556"&gt;Support comparison operations on Indexes #556&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/568"&gt;Support byte ranges in read_csv (and other formats) #568&lt;/a&gt;:w&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/824"&gt;Allow “df.index = some_index” #824&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/828"&gt;Support indexing on groupby objects #828&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/831"&gt;Support df.reset_index(drop=True) #831&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/879"&gt;Add Series.groupby #879&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/880"&gt;Support Dataframe/Series groupby level=0 #880&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/900"&gt;Implement division on DataFrame objects #900&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/934"&gt;Groupby objects aren’t indexable by column names #934&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/937"&gt;Support comparisons on index operations #937&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/944"&gt;Add DataFrame.rename #944&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/967"&gt;Set the index of a dataframe/series #967&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/968"&gt;Support concat(…, axis=1) #968&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/969"&gt;Support indexing with a pandas index from columns #969&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/970"&gt;Support indexing a dataframe with another boolean dataframe #970&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;I don’t really expect anyone to go through all of those issues, but my hope is
that by skimming over the issue titles people will get a sense for the kinds of
changes we’re making here. It’s a large number of small things.&lt;/p&gt;
&lt;p&gt;Also, kudos to &lt;a class="reference external" href="https://github.com/thomcom"&gt;Thomson Comer&lt;/a&gt; who solved most of
the cuDF issues above.&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/2019/03/04/building-gpu-groupbys.md&lt;/span&gt;, line 83)&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="there-are-still-some-pending-issues"&gt;
&lt;h1&gt;There are still some pending issues&lt;/h1&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/1055"&gt;Square Root #1055&lt;/a&gt;, needed for groupby-std&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/rapidsai/cudf/issues/483"&gt;cuDF needs multi-index support for columns #483&lt;/a&gt;, needed for:&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;gropuby&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;agg&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;sum&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;], &amp;#39;&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;: [&amp;#39;&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;, &amp;#39;&lt;/span&gt;&lt;span class="nb"&gt;max&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;]})&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/2019/03/04/building-gpu-groupbys.md&lt;/span&gt;, line 92)&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="but-things-mostly-work"&gt;
&lt;h1&gt;But things mostly work&lt;/h1&gt;
&lt;p&gt;But generally things work pretty well today:&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;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&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_cudf&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2&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;dask_cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;yellow_tripdata_2016-*.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;3&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;groupby&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;passenger_count&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;trip_distance&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;Out&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="o"&gt;&amp;lt;&lt;/span&gt;&lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Series&lt;/span&gt; &lt;span class="n"&gt;nrows&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_pandas&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="n"&gt;Out&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;    &lt;span class="mf"&gt;0.625424&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;    &lt;span class="mf"&gt;4.976895&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;    &lt;span class="mf"&gt;4.470014&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;    &lt;span class="mf"&gt;5.955262&lt;/span&gt;
&lt;span class="mi"&gt;4&lt;/span&gt;    &lt;span class="mf"&gt;4.328076&lt;/span&gt;
&lt;span class="mi"&gt;5&lt;/span&gt;    &lt;span class="mf"&gt;3.079661&lt;/span&gt;
&lt;span class="mi"&gt;6&lt;/span&gt;    &lt;span class="mf"&gt;2.998077&lt;/span&gt;
&lt;span class="mi"&gt;7&lt;/span&gt;    &lt;span class="mf"&gt;3.147452&lt;/span&gt;
&lt;span class="mi"&gt;8&lt;/span&gt;    &lt;span class="mf"&gt;5.165570&lt;/span&gt;
&lt;span class="mi"&gt;9&lt;/span&gt;    &lt;span class="mf"&gt;5.916169&lt;/span&gt;
&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;float64&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/2019/03/04/building-gpu-groupbys.md&lt;/span&gt;, line 119)&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="experience"&gt;
&lt;h1&gt;Experience&lt;/h1&gt;
&lt;p&gt;First, most of this work was handled by the cuDF developers (which may be
evident from the relative lengths of the issue lists above). When we started
this process it felt like a never-ending stream of tiny issues. We weren’t
able to see the next set of issues until we had finished the current set.
Fortunately, most of them were pretty easy to fix. Additionally, as we went
on, it seemed to get a bit easier over time.&lt;/p&gt;
&lt;p&gt;Additionally, lots of things work other than groupby-aggregations as a result
of the changes above. From the perspective of someone accustomed to Pandas,
The cuDF library is starting to feel more reliable. We hit missing
functionality less frequently when using cuDF on other operations.&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/2019/03/04/building-gpu-groupbys.md&lt;/span&gt;, line 133)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="what-s-next"&gt;
&lt;h1&gt;What’s next?&lt;/h1&gt;
&lt;p&gt;More recently we’ve been working on the various join/merge operations in Dask
DataFrame like indexed joins on a sorted column, joins between large and small
dataframes (a common special case) and so on. Getting these algorithms from
the mainline Dask DataFrame codebase to work with cuDF is resulting in a
similar set of issues to what we saw above with groupby-aggregations, but so
far the list is much smaller. We hope that this is a trend as we continue on
to other sets of functionality into the future like I/O, time-series
operations, rolling windows, and so on.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/03/04/building-gpu-groupbys/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="GPU" label="GPU"/>
    <category term="RAPIDS" label="RAPIDS"/>
    <category term="dataframe" label="dataframe"/>
    <published>2019-03-04T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/01/29/cudf-joins/</id>
    <title>Single-Node Multi-GPU Dataframe Joins</title>
    <updated>2019-01-29T00:00:00+00:00</updated>
    <author>
      <name>Matthew Rocklin</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/2019/01/29/cudf-joins.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;We experiment with single-node multi-GPU joins using cuDF and Dask. We find
that the in-GPU computation is faster than communication. We also present
context and plans for near-future work, including improving high performance
communication in Dask with UCX.&lt;/p&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/6e2c33c33b32bc324e3965212f202f66"&gt;Here is a notebook of the experiment in this post&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/2019/01/29/cudf-joins.md&lt;/span&gt;, line 18)&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="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;In a recent post we showed how Dask + cuDF could accelerate reading CSV files
using multiple GPUs in parallel. That operation quickly became bound by the
speed of our disk after we added a few GPUs. Now we try a very different kind
of operation, multi-GPU joins.&lt;/p&gt;
&lt;p&gt;This workload can be communication-heavy, especially if the column on which we
are joining is not sorted nicely, and so provides a good example on the other
extreme from parsing CSV.&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/2019/01/29/cudf-joins.md&lt;/span&gt;, line 29)&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="benchmark"&gt;
&lt;h1&gt;Benchmark&lt;/h1&gt;
&lt;section id="construct-random-data-using-the-cpu"&gt;
&lt;h2&gt;Construct random data using the CPU&lt;/h2&gt;
&lt;p&gt;Here we use Dask array and Dask dataframe to construct two random tables with a
shared &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;id&lt;/span&gt;&lt;/code&gt; column. We can play with the number of rows of each table and the
number of keys to make the join challenging in a variety of ways.&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.array&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;da&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;n_rows&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1000000000&lt;/span&gt;
&lt;span class="n"&gt;n_keys&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;5000000&lt;/span&gt;

&lt;span class="n"&gt;left&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;concat&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;
    &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_rows&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_dask_dataframe&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;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
    &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_keys&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;n_rows&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_dask_dataframe&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;id&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;n_rows&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;10000000&lt;/span&gt;

&lt;span class="n"&gt;right&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;concat&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;
    &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_rows&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_dask_dataframe&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;y&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
    &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;randint&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_keys&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;n_rows&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_dask_dataframe&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;id&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="send-to-the-gpus"&gt;
&lt;h2&gt;Send to the GPUs&lt;/h2&gt;
&lt;p&gt;We have two Dask dataframes composed of many Pandas dataframes of our random
data. We now map the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cudf.from_pandas&lt;/span&gt;&lt;/code&gt; function across these to make a Dask
dataframe of cuDF dataframes.&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&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;cudf&lt;/span&gt;

&lt;span class="n"&gt;gleft&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;left&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;gright&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;right&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map_partitions&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cudf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_pandas&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;gleft&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gright&lt;/span&gt; &lt;span class="o"&gt;=&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;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;gleft&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;gright&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# persist data in device memory&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;What’s nice here is that there wasn’t any special
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask_pandas_dataframe_to_dask_cudf_dataframe&lt;/span&gt;&lt;/code&gt; function. Dask composed nicely
with cuDF. We didn’t need to do anything special to support it.&lt;/p&gt;
&lt;p&gt;We’ll also persisted the data in device memory.&lt;/p&gt;
&lt;p&gt;After this, simple operations are easy and fast and use our eight GPUs.&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;gleft&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;x&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# this takes 250ms&lt;/span&gt;
&lt;span class="go"&gt;500004719.254711&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="join"&gt;
&lt;h2&gt;Join&lt;/h2&gt;
&lt;p&gt;We’ll use standard Pandas syntax to merge the datasets, persist the result in
RAM, and then wait&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;out&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;gleft&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;merge&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;gright&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;on&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;id&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;  &lt;span class="c1"&gt;# this is lazy&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/2019/01/29/cudf-joins.md&lt;/span&gt;, line 95)&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="profile-and-analyze-results"&gt;
&lt;h1&gt;Profile and analyze results&lt;/h1&gt;
&lt;p&gt;We now look at the Dask diagnostic plots for this computation.&lt;/p&gt;
&lt;section id="task-stream-and-communication"&gt;
&lt;h2&gt;Task stream and communication&lt;/h2&gt;
&lt;p&gt;When we look at Dask’s task stream plot we see that each of our eight threads
(each of which manages a single GPU) spent most of its time in communication
(red is communication time). The actual merge and concat tasks are quite fast
relative to the data transfer time.&lt;/p&gt;
&lt;iframe src="https://matthewrocklin.com/raw-host/dask-cudf-joins.html"
        width="800"
        height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;That’s not too surprising. For this computation I’ve turned off any attempt to
communicate between devices (more on this below) so the data is being moved
from the GPU to the CPU memory, then serialized and put onto a TCP socket.
We’re moving tens of GB on a single machine, so we’re seeing about 1GB/s total
throughput of the system, which is typical for TCP-on-localhost in Python.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="flamegraph-of-computation"&gt;
&lt;h2&gt;Flamegraph of computation&lt;/h2&gt;
&lt;p&gt;We can also look more deeply at the computational costs in Dask’s
flamegraph-style plot. This shows which lines of our functions were taking up
the most time (down to the Python level at least).&lt;/p&gt;
&lt;iframe src="http://matthewrocklin.com/raw-host/dask-cudf-join-profile.html"
        width="800"
        height="400"&gt;&lt;/iframe&gt;
&lt;p&gt;This &lt;a class="reference external" href="http://www.brendangregg.com/flamegraphs.html"&gt;Flame graph&lt;/a&gt; shows which
lines of cudf code we spent time on while computing (excluding the main
communication costs mentioned above). It may be interesting for those trying
to further optimize performance. It shows that most of our costs are in memory
allocation. Like communication, this has actually also been fixed in RAPIDS’
optional memory management pool, it just isn’t default yet, so I didn’t use it
here.&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/2019/01/29/cudf-joins.md&lt;/span&gt;, line 134)&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="plans-for-efficient-communication"&gt;
&lt;h1&gt;Plans for efficient communication&lt;/h1&gt;
&lt;p&gt;The cuDF library actually has a decent approach to single-node multi-GPU
communication that I’ve intentionally turned off for this experiment. That
approach cleverly used Dask to communicate device pointer information using
Dask’s normal channels (this is small and fast) and then used that information
to initiate a side-channel communication for the bulk of the data. This
approach was effective, but somewhat fragile. I’m inclined to move on for it
in favor of …&lt;/p&gt;
&lt;p&gt;UCX. The &lt;a class="reference external" href="http://www.openucx.org/"&gt;UCX&lt;/a&gt; project provides a single API that
wraps around several transports like TCP, Infiniband, shared memory, and also
GPU-specific transports. UCX claims to find the best way to communicate data
between two points given the hardware available. If Dask were able to use this
for communication then it would provide both efficient GPU-to-GPU communication
on a single machine, and also efficient inter-machine communication when
efficient networking hardware like Infiniband was present, even outside the
context of GPUs.&lt;/p&gt;
&lt;p&gt;There is some work we need to do here:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We need to make a Python wrapper around UCX&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We need to make an optional &lt;a class="reference external" href="https://distributed.dask.org/en/latest/communications.html"&gt;Dask Comm&lt;/a&gt;
around this ucx-py library that allows users to specify endpoints like
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ucx://path-to-scheduler&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We need to make Python memoryview-like objects that refer to device memory&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;…&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This work is already in progress by &lt;a class="reference external" href="https://github.com/Akshay-Venkatesh"&gt;Akshay
Vekatesh&lt;/a&gt;, who works on UCX, and &lt;a class="reference external" href="https://tomaugspurger.github.io/"&gt;Tom
Augspurger&lt;/a&gt; a core Dask/Pandas developer. I
suspect that they’ll write about it soon. I’m looking forward to seeing what
comes of it, both for Dask and for high performance Python generally.&lt;/p&gt;
&lt;p&gt;It’s worth pointing out that this effort won’t just help GPU users. It should
help anyone on advanced networking hardware, including the mainstream
scientific HPC community.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/29/cudf-joins.md&lt;/span&gt;, line 172)&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="id1"&gt;
&lt;h1&gt;Summary&lt;/h1&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: INFO/1 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/01/29/cudf-joins.md&lt;/span&gt;, line 172); &lt;em&gt;&lt;a href="#id1"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “summary”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Single-node Mutli-GPU joins have a lot of promise. In fact, earlier RAPIDS
developers got this running much faster than I was able to do above through the
clever communication tricks I briefly mentioned. The main purpose of this post
is to provide a benchmark for joins that we can use in the future, and to
highlight when communication can be essential in parallel computing.&lt;/p&gt;
&lt;p&gt;Now that GPUs have accelerated the computation time of each of our chunks of
work we increasingly find that other systems become the bottleneck. We didn’t
care as much about communication before because computational costs were
comparable. Now that computation is an order of magnitude cheaper, other
aspects of our stack become much more important.&lt;/p&gt;
&lt;p&gt;I’m looking forward to seeing where this goes.&lt;/p&gt;
&lt;section id="come-help"&gt;
&lt;h2&gt;Come help!&lt;/h2&gt;
&lt;p&gt;If the work above sounds interesting to you then come help!
There is a lot of low-hanging and high impact work to do.&lt;/p&gt;
&lt;p&gt;If you’re interested in being paid to focus more on these topics, then consider
applying for a job. NVIDIA’s RAPIDS team is looking to hire engineers for Dask
development with GPUs and other data analytics library development projects.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://nvidia.wd5.myworkdayjobs.com/en-US/NVIDIAExternalCareerSite/job/US-TX-Austin/Senior-Library-Software-Engineer---RAPIDS_JR1919608-1"&gt;Senior Library Software Engineer - RAPIDS&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/01/29/cudf-joins/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="GPU" label="GPU"/>
    <category term="dataframe" label="dataframe"/>
    <published>2019-01-29T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/01/22/dask-extension-arrays/</id>
    <title>Extension Arrays in Dask DataFrame</title>
    <updated>2019-01-22T00:00:00+00:00</updated>
    <author>
      <name>Tom Augspurger</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://anaconda.com"&gt;Anaconda Inc&lt;/a&gt;&lt;/em&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/2019/01/22/dask-extension-arrays.md&lt;/span&gt;, line 11)&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;Dask DataFrame works well with pandas’ new Extension Array interface, including
third-party extension arrays. This lets Dask&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;easily support pandas’ new extension arrays, like their new &lt;a class="reference external" href="http://pandas.pydata.org/pandas-docs/version/0.24/whatsnew/v0.24.0.html#optional-integer-na-support"&gt;nullable integer
array&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;support third-party extension array arrays, like &lt;a class="reference external" href="https://cyberpandas.readthedocs.io"&gt;cyberpandas’s&lt;/a&gt;
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;IPArray&lt;/span&gt;&lt;/code&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/2019/01/22/dask-extension-arrays.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&gt;
&lt;section id="background"&gt;
&lt;h1&gt;Background&lt;/h1&gt;
&lt;p&gt;Pandas 0.23 introduced the &lt;a class="reference external" href="http://pandas.pydata.org/pandas-docs/version/0.24/extending.html#extension-types"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ExtensionArray&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;, a way to store things other
than a simple NumPy array in a DataFrame or Series. Internally pandas uses this
for data types that aren’t handled natively by NumPy like datetimes with
timezones, Categorical, or (the new!) nullable integer arrays.&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Series&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;date_range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;2000&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;periods&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;tz&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;US/Central&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;
&lt;span class="go"&gt;0   2000-01-01 00:00:00-06:00&lt;/span&gt;
&lt;span class="go"&gt;1   2000-01-02 00:00:00-06:00&lt;/span&gt;
&lt;span class="go"&gt;2   2000-01-03 00:00:00-06:00&lt;/span&gt;
&lt;span class="go"&gt;3   2000-01-04 00:00:00-06:00&lt;/span&gt;
&lt;span class="go"&gt;dtype: datetime64[ns, US/Central]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; has always supported the extension types that pandas defines.&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="gp"&gt;&amp;gt;&amp;gt;&amp;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="gp"&gt;&amp;gt;&amp;gt;&amp;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;from_pandas&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;npartitions&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;Dask Series Structure:&lt;/span&gt;
&lt;span class="go"&gt;npartitions=2&lt;/span&gt;
&lt;span class="go"&gt;0    datetime64[ns, US/Central]&lt;/span&gt;
&lt;span class="go"&gt;2                           ...&lt;/span&gt;
&lt;span class="go"&gt;3                           ...&lt;/span&gt;
&lt;span class="go"&gt;dtype: datetime64[ns, US/Central]&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: from_pandas, 2 tasks&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/2019/01/22/dask-extension-arrays.md&lt;/span&gt;, line 52)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-challenge"&gt;
&lt;h1&gt;The Challenge&lt;/h1&gt;
&lt;p&gt;Newer versions of pandas allow third-party libraries to write custom extension
arrays. These arrays can be placed inside a DataFrame or Series, and work
just as well as any extension array defined within pandas itself. However,
third-party extension arrays provide a slight challenge for Dask.&lt;/p&gt;
&lt;p&gt;Recall: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; is lazy. We use a familiar pandas-like API to build up
a task graph, rather than executing immediately. But if Dask DataFrame is lazy,
then how do things like the following work?&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;A&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;B&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;]})&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&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;from_pandas&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;npartitions&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="p"&gt;[[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;B&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;
&lt;span class="go"&gt;Index([&amp;#39;B&amp;#39;], dtype=&amp;#39;object&amp;#39;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ddf[['B']]&lt;/span&gt;&lt;/code&gt; (lazily) selects the column &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;'B'&lt;/span&gt;&lt;/code&gt; from the dataframe. But accessing
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.columns&lt;/span&gt;&lt;/code&gt; &lt;em&gt;immediately&lt;/em&gt; returns a pandas Index object with just the selected
columns.&lt;/p&gt;
&lt;p&gt;No real computation has happened (you could just as easily swap out the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_pandas&lt;/span&gt;&lt;/code&gt; for a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dd.read_parquet&lt;/span&gt;&lt;/code&gt; on a larger-than-memory dataset, and the
behavior would be the same). Dask is able to do these kinds of “metadata-only”
computations, where the output depends only on the columns and the dtypes,
without executing the task graph. Internally, Dask does this by keeping a pair
of dummy pandas DataFrames on each Dask DataFrame.&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_meta&lt;/span&gt;
&lt;span class="go"&gt;Empty DataFrame&lt;/span&gt;
&lt;span class="go"&gt;Columns: [A, B]&lt;/span&gt;
&lt;span class="go"&gt;Index: []&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_meta_nonempty&lt;/span&gt;
&lt;span class="go"&gt;ddf._meta_nonempty&lt;/span&gt;
&lt;span class="go"&gt;   A  B&lt;/span&gt;
&lt;span class="go"&gt;0  1  1&lt;/span&gt;
&lt;span class="go"&gt;1  1  1&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We need the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_meta_nonempty&lt;/span&gt;&lt;/code&gt;, since some operations in pandas behave differently
on an Empty DataFrame than on a non-empty one (either by design or,
occasionally, a bug in pandas).&lt;/p&gt;
&lt;p&gt;The issue with third-party extension arrays is that Dask doesn’t know what
values to put in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_meta_nonempty&lt;/span&gt;&lt;/code&gt;. We’re quite happy to do it for each NumPy
dtype and each of pandas’ own extension dtypes. But any third-party library
could create an ExtensionArray for any type, and Dask would have no way of
knowing what’s a valid value for it.&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/2019/01/22/dask-extension-arrays.md&lt;/span&gt;, line 104)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-solution"&gt;
&lt;h1&gt;The Solution&lt;/h1&gt;
&lt;p&gt;Rather than Dask guessing what values to use for the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_meta_nonempty&lt;/span&gt;&lt;/code&gt;, extension
array authors (or users) can register their extension dtype with Dask. Once
registered, Dask will be able to generate the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;_meta_nonempty&lt;/span&gt;&lt;/code&gt;, and things
should work fine from there. For example, we can register the dummy &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DecimalArray&lt;/span&gt;&lt;/code&gt;
that pandas uses for testing (this isn’t part of pandas’ public API) with Dask.&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;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;decimal&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Decimal&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pandas.tests.extension.decimal&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;DecimalArray&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;DecimalDtype&lt;/span&gt;

&lt;span class="c1"&gt;# The actual registration that would be done in the 3rd-party library&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe.extensions&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;make_array_nonempty&lt;/span&gt;


&lt;span class="nd"&gt;@make_array_nonempty&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;DecimalDtype&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;_&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;DecimalArray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;_from_sequence&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;NaN&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)],&lt;/span&gt;
                                       &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now users of that extension type can place those arrays inside a Dask DataFrame
or Series.&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;A&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;DecimalArray&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;1.0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;2.0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;                                      &lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;3.0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)])})&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&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;from_pandas&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="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;
&lt;span class="go"&gt;Dask DataFrame Structure:&lt;/span&gt;
&lt;span class="go"&gt;                     A&lt;/span&gt;
&lt;span class="go"&gt;npartitions=1&lt;/span&gt;
&lt;span class="go"&gt;0              decimal&lt;/span&gt;
&lt;span class="go"&gt;2                  ...&lt;/span&gt;
&lt;span class="go"&gt;Dask Name: from_pandas, 1 tasks&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dtypes&lt;/span&gt;
&lt;span class="go"&gt;A    decimal&lt;/span&gt;
&lt;span class="go"&gt;dtype: object&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And from there, the usual operations just as they would in pandas.&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;random&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;choices&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;DataFrame&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;A&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;DecimalArray&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;choices&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;1.0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;                                              &lt;span class="n"&gt;Decimal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;2.0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)],&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;                                             &lt;span class="n"&gt;k&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;                   &lt;span class="s2"&gt;&amp;quot;B&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;choice&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;,))})&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;ddf&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;from_pandas&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="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;In [35]: ddf.groupby(&amp;quot;A&amp;quot;).B.mean().compute()&lt;/span&gt;
&lt;span class="go"&gt;Out[35]:&lt;/span&gt;
&lt;span class="go"&gt;A&lt;/span&gt;
&lt;span class="go"&gt;1.0    1.50&lt;/span&gt;
&lt;span class="go"&gt;2.0    1.48&lt;/span&gt;
&lt;span class="go"&gt;Name: B, dtype: float64&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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/2019/01/22/dask-extension-arrays.md&lt;/span&gt;, line 165)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-real-lesson"&gt;
&lt;h1&gt;The Real Lesson&lt;/h1&gt;
&lt;p&gt;It’s neat that Dask now supports extension arrays. But to me, the exciting thing
is just how little work this took. The
&lt;a class="reference external" href="https://github.com/dask/dask/pull/4379/files"&gt;PR&lt;/a&gt; implementing support for
third-party extension arrays is quite short, just defining the object that
third-parties register with, and using it to generate the data when dtype is
detected. Supporting the three new extension arrays in pandas 0.24.0
(&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;IntegerArray&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;PeriodArray&lt;/span&gt;&lt;/code&gt;, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;IntervalArray&lt;/span&gt;&lt;/code&gt;), takes a handful of lines
of code&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="nd"&gt;@make_array_nonempty&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Interval&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;_&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;IntervalArray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_breaks&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;closed&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;closed&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;


&lt;span class="nd"&gt;@make_array_nonempty&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Period&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;_&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;period_array&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;2000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2001&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;freq&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;freq&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;


&lt;span class="nd"&gt;@make_array_nonempty&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;register&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;_IntegerDtype&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;_&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;integer_array&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask benefits directly from improvements made to pandas. Dask didn’t have to
build out a new parallel extension array interface, and reimplement all the new
extension arrays using the parallel interface. We just re-used what pandas
already did, and it fits into the existing Dask structure.&lt;/p&gt;
&lt;p&gt;For third-party extension array authors, like &lt;a class="reference external" href="https://cyberpandas.readthedocs.io"&gt;cyberpandas&lt;/a&gt;, the
work is similarly minimal. They don’t need to re-implement everything from the
ground up, just to play well with Dask.&lt;/p&gt;
&lt;p&gt;This highlights the importance of one of the Dask project’s core values: working
with the community. If you visit &lt;a class="reference external" href="https://dask.org"&gt;dask.org&lt;/a&gt;, you’ll see
phrases like&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Integrates with existing projects&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;and&lt;/p&gt;
&lt;blockquote&gt;
&lt;div&gt;&lt;p&gt;Built with the broader community&lt;/p&gt;
&lt;/div&gt;&lt;/blockquote&gt;
&lt;p&gt;At the start of Dask, the developers &lt;em&gt;could&lt;/em&gt; have gone off and re-written pandas
or NumPy from scratch to be parallel friendly (though we’d probably still be
working on that part today, since that’s such a massive undertaking). Instead,
the Dask developers worked with the community, occasionally nudging it in
directions that would help out dask. For example, many places in pandas &lt;a class="reference external" href="http://matthewrocklin.com/blog/work/2015/03/10/PyData-GIL"&gt;held
the GIL&lt;/a&gt;, preventing
thread-based parallelism. Rather than abandoning pandas, the Dask and pandas
developers worked together to release the GIL where possible when it was a
bottleneck for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt;. This benefited Dask and anyone else trying to
do thread-based parallelism with pandas DataFrames.&lt;/p&gt;
&lt;p&gt;And now, when pandas introduces new features like nullable integers,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; just needs to register it as an extension type and immediately
benefits from it. And third-party extension array authors can do the same for
their extension arrays.&lt;/p&gt;
&lt;p&gt;If you’re writing an ExtensionArray, make sure to add it to the &lt;a class="reference external" href="http://pandas.pydata.org/pandas-docs/version/0.24/ecosystem.html#extension-data-types"&gt;pandas
ecosystem&lt;/a&gt; page, and register it with Dask!&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/01/22/dask-extension-arrays/"/>
    <summary>This work is supported by Anaconda Inc</summary>
    <category term="dask" label="dask"/>
    <category term="dataframe" label="dataframe"/>
    <published>2019-01-22T00:00:00+00:00</published>
  </entry>
</feed>
