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

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

&lt;span class="c1"&gt;# The dataframe.shuffle.method config is available since 2023.3.1&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;set&lt;/span&gt;&lt;span class="p"&gt;({&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dataframe.shuffle.method&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;tasks&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;}):&lt;/span&gt;
    &lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;...&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# many cluster managers send current dask config automatically&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For more information on deactivating P2P shuffle, see the &lt;a class="reference external" href="https://github.com/dask/distributed/discussions/7509"&gt;discussion #7509&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 152)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="what-about-arrays"&gt;
&lt;h1&gt;What about arrays?&lt;/h1&gt;
&lt;p&gt;While the original motivation was to optimize large-scale dataframe joins, the P2P system is useful for all problems requiring lots of communication between tasks.
For array workloads, this often occurs when rechunking data, such as when organizing a matrix by rows when it was stored by columns.
Similar to dataframe joins, array rechunking has been inefficient in the past, which has become such a problem that the array community built specialized tools like &lt;a class="reference external" href="https://github.com/pangeo-data/rechunker"&gt;rechunker&lt;/a&gt; to avoid it entirely.&lt;/p&gt;
&lt;p&gt;There is a &lt;a class="reference external" href="https://github.com/dask/distributed/pull/7534"&gt;naive implementation&lt;/a&gt; of array rechunking using the P2P system available for experimental use.
Benchmarking this implementation shows mixed results:&lt;/p&gt;
&lt;figure class="align-center"&gt;
&lt;img alt="P2P rechunking uses constant memory." src="/images/p2p_benchmark_arrays.png" style="width: 100%;"/&gt;
&lt;/figure&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;👍 &lt;strong&gt;Constant memory use:&lt;/strong&gt; As with dataframe operations, memory use is constant.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;❓ &lt;strong&gt;Variable runtime:&lt;/strong&gt; The runtime of workloads may increase with P2P.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;👎 &lt;strong&gt;Memory overhead:&lt;/strong&gt; There is a large memory overhead for many small partitions.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The constant memory use is a very promising result.
There are several ways to tackle the current limitations.
We expect this to improve as we work with collaborators from the array computing community.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 173)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="next-steps"&gt;
&lt;h1&gt;Next steps&lt;/h1&gt;
&lt;p&gt;Development on P2P shuffling is not done yet.
For the future we plan the following:&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt;&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;While the early prototype of array rechunking is promising, it’s not there yet.
We plan to do the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Intelligently select which algorithm to use (task-based rechunking is better sometimes)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Work with collaborators on rechunking to improve performance&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Seek out other use cases, like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map_overlap&lt;/span&gt;&lt;/code&gt;, where this might be helpful&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Failure recovery&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;Make P2P shuffling resilient to worker loss; currently, it has to restart entirely.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Performance tuning&lt;/strong&gt;:&lt;/p&gt;
&lt;p&gt;Performance today is good, but not yet at peak hardware speeds.
We can improve this in a few ways:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Hiding disk I/O&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Using more memory when appropriate for smaller shuffles&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Improved batching of small operations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;More efficient serialization&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;To follow along with the development, subscribe to the &lt;a class="reference external" href="https://github.com/dask/distributed/issues/7507"&gt;tracking issue&lt;/a&gt; on GitHub.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/03/15/shuffling-large-data-at-constant-memory.md&lt;/span&gt;, line 203)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="summary"&gt;
&lt;h1&gt;Summary&lt;/h1&gt;
&lt;p&gt;Shuffling data is a common operation in dataframe workloads.
Since &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;2023.2.1&lt;/span&gt;&lt;/code&gt;, Dask defaults to P2P shuffling for distributed clusters and shuffles data faster and at constant memory.
This improvement unlocks previously un-runnable workload sizes and efficiently uses your clusters.
Finally, P2P shuffling demonstrates extending Dask to add new paradigms while leveraging the foundations of its distributed engine.&lt;/p&gt;
&lt;p&gt;Share results in &lt;a class="reference external" href="https://github.com/dask/distributed/issues/7507"&gt;this discussion thread&lt;/a&gt; or follow development at this &lt;a class="reference external" href="https://github.com/dask/distributed/issues/7507"&gt;tracking issue&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/03/15/shuffling-large-data-at-constant-memory/"/>
    <summary>This work was engineered and supported by Coiled. In particular, thanks to Florian Jetter, Gabe Joseph, Hendrik Makait, and Matt Rocklin. Original version of this post appears on blog.coiled.io</summary>
    <category term="dask" label="dask"/>
    <category term="distributed" label="distributed"/>
    <category term="p2p" label="p2p"/>
    <category term="shuffling" label="shuffling"/>
    <published>2023-03-15T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2022/07/19/dask-multi-cloud/</id>
    <title>Data Proximate Computation on a Dask Cluster Distributed Between Data Centres</title>
    <updated>2022-07-19T00:00:00+00:00</updated>
    <author>
      <name>Richard Care (Met Office)</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is a joint venture between the &lt;a class="reference external" href="https://www.metoffice.gov.uk"&gt;Met Office&lt;/a&gt; and the &lt;a class="reference external" href="https://www.europeanweather.cloud/"&gt;European Weather Cloud&lt;/a&gt;, which is a partnership of &lt;a class="reference external" href="https://ecmwf.int"&gt;ECMWF&lt;/a&gt; and &lt;a class="reference external" href="https://eumetsat.int/"&gt;EUMETSAT&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/2022/07/19/dask-multi-cloud.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;We have devised a technique for creating a Dask cluster where worker nodes are hosted in different data centres, connected by a mesh VPN that allows the scheduler and workers to communicate and exchange results.&lt;/p&gt;
&lt;p&gt;A novel (ab)use of Dask resources allows us to run data processing tasks on the workers in the cluster closest to the source data, so that communication between data centres is minimised. If combined with zarr to give access to huge hyper-cube datasets in object storage, we believe that the technique could realise the potential of data-proximate distributed computing in the Cloud.&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/07/19/dask-multi-cloud.md&lt;/span&gt;, line 17)&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;The UK Met Office has been carrying out a study into data-proximate computing in collaboration with the European Weather Cloud. We identified Dask as a key technology, but whilst existing Dask techniques focus on parallel computation in a single data centre, we looked to extend this to computation across data centres. This way, when the data required is hosted in multiple locations, tasks can be run where the data is rather than copying it.&lt;/p&gt;
&lt;p&gt;Dask worker nodes exchange data chunks over a network, coordinated by a scheduler. There is an assumption that all nodes are freely able to communicate, which is not generally true across data centres due to firewalls, NAT, etc, so a truly distributed approach has to solve this problem. In addition, it has to manage data transfer efficiently, because moving data chunks between data centres is much more costly than between workers in the same cloud.&lt;/p&gt;
&lt;p&gt;This notebook documents a running proof-of-concept that addresses these problems. It runs a computation in 3 locations:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;This computer, where the client and scheduler are running. This was run on AWS during development.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The ECMWF data centre. This has compute resources, and hosts data containing &lt;em&gt;predictions&lt;/em&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The EUMETSAT data centre, with compute resources and data on &lt;em&gt;observations&lt;/em&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&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;IPython.display&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;Image&lt;/span&gt;
&lt;span class="n"&gt;Image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;filename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;images/datacentres.png&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="c1"&gt;# this because GitHub doesn&amp;#39;t render markup images in private repos&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/dask-multi-cloud_1_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;The idea is that tasks accessing data available in a location should be run there. Meanwhile the computation can be defined, invoked, and the results rendered, elsewhere. All this with minimal hinting to the computation as to how this should be done.&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/07/19/dask-multi-cloud.md&lt;/span&gt;, line 38)&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="setup"&gt;
&lt;h1&gt;Setup&lt;/h1&gt;
&lt;p&gt;First some imports and conveniences&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;os&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;time&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;sleep&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;performance_report&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;get_task_stream&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_worker_pools&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;pool&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;propagate_pools&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;pytest&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;ipytest&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;xarray&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;matplotlib.pyplot&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;plt&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;orgs&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;my_org&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;tree&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;tree&lt;/span&gt;

&lt;span class="n"&gt;ipytest&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;autoconfig&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In this case we are using a control plane IPv4 &lt;a class="reference external" href="https://www.wireguard.com/"&gt;WireGuard&lt;/a&gt; network on 10.8.0.0/24 to set up the cluster - this is not a necessity, but simplifies this proof of concept. WireGuard peers are running on ECMWF and EUMETSAT machines already, but we have to start one here:&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="err"&gt;!&lt;/span&gt;&lt;span class="o"&gt;./&lt;/span&gt;&lt;span class="n"&gt;start&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;wg&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sh&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;4: mo-aws-ec2: &amp;lt;POINTOPOINT,NOARP,UP,LOWER_UP&amp;gt; mtu 8921 qdisc noqueue state UNKNOWN group default qlen 1000
    link/none
    inet 10.8.0.3/24 scope global mo-aws-ec2
       valid_lft forever preferred_lft forever
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We have worker machines configured in both ECMWF and EUMETSAT, one in each. They are accessible on the control plane network as&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;ecmwf_host&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;10.8.0.4&amp;#39;&lt;/span&gt;
&lt;span class="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;env&lt;/span&gt; &lt;span class="n"&gt;ECMWF_HOST&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="err"&gt;$&lt;/span&gt;&lt;span class="n"&gt;ecmwf_host&lt;/span&gt;
&lt;span class="n"&gt;eumetsat_host&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;10.8.0.2&amp;#39;&lt;/span&gt;
&lt;span class="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;env&lt;/span&gt; &lt;span class="n"&gt;EUMETSAT_HOST&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="err"&gt;$&lt;/span&gt;&lt;span class="n"&gt;eumetsat_host&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;env: ECMWF_HOST=10.8.0.4
env: EUMETSAT_HOST=10.8.0.2
&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/07/19/dask-multi-cloud.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="mount-the-data"&gt;
&lt;h1&gt;Mount the Data&lt;/h1&gt;
&lt;p&gt;This machine needs access to the data files over the network in order to read NetCDF metadata. The workers are sharing their data with NFS, so we mount them here. (In this proof of concept, the control plane network is used for NFS, but the data plane network could equally be used, or a more appropriate technology such as &lt;a class="reference external" href="https://zarr.readthedocs.io/en/stable/"&gt;zarr&lt;/a&gt; accessing object storage.)&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;%%bash
sudo&lt;span class="w"&gt; &lt;/span&gt;./data-reset.sh

mkdir&lt;span class="w"&gt; &lt;/span&gt;-p&lt;span class="w"&gt; &lt;/span&gt;/data/ecmwf
mkdir&lt;span class="w"&gt; &lt;/span&gt;-p&lt;span class="w"&gt; &lt;/span&gt;/data/eumetsat
sudo&lt;span class="w"&gt; &lt;/span&gt;mount&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nv"&gt;$ECMWF_HOST&lt;/span&gt;:/data/ecmwf&lt;span class="w"&gt; &lt;/span&gt;/data/ecmwf
sudo&lt;span class="w"&gt; &lt;/span&gt;mount&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nv"&gt;$EUMETSAT_HOST&lt;/span&gt;:/eumetsatdata/&lt;span class="w"&gt; &lt;/span&gt;/data/eumetsat
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&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;Image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;filename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;images/datacentres-data.png&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;&lt;img alt="png" src="https://blog.dask.org/_images/dask-multi-cloud_11_0.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 102)&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="access-to-data"&gt;
&lt;h1&gt;Access to Data&lt;/h1&gt;
&lt;p&gt;For this demonstration, we have two large data files that we want to process. On ECMWF we have predictions in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;/data/ecmwf/000490262cdd067721a34112963bcaa2b44860ab.nc&lt;/span&gt;&lt;/code&gt;. Workers running in ECMWF can see the file&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="err"&gt;!&lt;/span&gt;&lt;span class="n"&gt;ssh&lt;/span&gt; &lt;span class="o"&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;ssh&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;id_rsa_rcar_infra&lt;/span&gt;  &lt;span class="n"&gt;rcar&lt;/span&gt;&lt;span class="o"&gt;@&lt;/span&gt;&lt;span class="err"&gt;$&lt;/span&gt;&lt;span class="n"&gt;ECMWF_HOST&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;tree /data/&amp;#39;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;/data/
└── ecmwf
    └── 000490262cdd067721a34112963bcaa2b44860ab.nc

1 directory, 1 file
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;and because that directory is mounted here over NFS, so can this computer&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="err"&gt;!&lt;/span&gt;&lt;span class="n"&gt;tree&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;ecmwf&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;/data/ecmwf
└── 000490262cdd067721a34112963bcaa2b44860ab.nc

0 directories, 1 file
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is a big file&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="err"&gt;!&lt;/span&gt;&lt;span class="n"&gt;ls&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;lh&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;ecmwf&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;total 2.8G
-rw-rw-r-- 1 ec2-user ec2-user 2.8G Mar 25 13:09 000490262cdd067721a34112963bcaa2b44860ab.nc
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;On EUMETSAT we have &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;observations.nc&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="err"&gt;!&lt;/span&gt;&lt;span class="n"&gt;ssh&lt;/span&gt; &lt;span class="o"&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;ssh&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;id_rsa_rcar_infra&lt;/span&gt;  &lt;span class="n"&gt;rcar&lt;/span&gt;&lt;span class="o"&gt;@&lt;/span&gt;&lt;span class="err"&gt;$&lt;/span&gt;&lt;span class="n"&gt;EUMETSAT_HOST&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;tree /data/eumetsat/ad-hoc&amp;#39;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;/data/eumetsat/ad-hoc
└── observations.nc

0 directories, 1 file
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;similarly visible on this computer&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="err"&gt;!&lt;/span&gt;&lt;span class="n"&gt;ls&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;lh&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;eumetsat&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;ad&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;hoc&lt;/span&gt;&lt;span class="o"&gt;/&lt;/span&gt;&lt;span class="n"&gt;observations&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;nc&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;-rw-rw-r-- 1 613600004 613600004 4.8M May 20 10:57 /data/eumetsat/ad-hoc/observations.nc
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Crucially, ECMWF data is not visible in the EUMETSAT data centre, and vice versa.&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/07/19/dask-multi-cloud.md&lt;/span&gt;, line 157)&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="our-calculation"&gt;
&lt;h1&gt;Our Calculation&lt;/h1&gt;
&lt;p&gt;We want to compare the predictions against the observations.&lt;/p&gt;
&lt;p&gt;We can open the predictions file with xarray&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;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/ecmwf/000490262cdd067721a34112963bcaa2b44860ab.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;predictions&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&amp;lt;xarray.Dataset&amp;gt;
Dimensions:                  (realization: 18, height: 33, latitude: 960,
longitude: 1280, bnds: 2)
Coordinates:
* realization              (realization) int32 0 18 19 20 21 ... 31 32 33 34
* height                   (height) float32 5.0 10.0 20.0 ... 5.5e+03 6e+03
* latitude                 (latitude) float32 -89.91 -89.72 ... 89.72 89.91
* longitude                (longitude) float32 -179.9 -179.6 ... 179.6 179.9
  forecast_period          timedelta64[ns] 1 days 18:00:00
  forecast_reference_time  datetime64[ns] 2021-11-07T06:00:00
  time                     datetime64[ns] 2021-11-09
  Dimensions without coordinates: bnds
  Data variables:
  air_pressure             (realization, height, latitude, longitude) float32 dask.array&amp;lt;chunksize=(18, 33, 192, 160), meta=np.ndarray&amp;gt;
  latitude_longitude       int32 -2147483647
  latitude_bnds            (latitude, bnds) float32 dask.array&amp;lt;chunksize=(960, 2), meta=np.ndarray&amp;gt;
  longitude_bnds           (longitude, bnds) float32 dask.array&amp;lt;chunksize=(1280, 2), meta=np.ndarray&amp;gt;
  Attributes:
  history:                      2021-11-07T10:27:38Z: StaGE Decoupler
  institution:                  Met Office
  least_significant_digit:      1
  mosg__forecast_run_duration:  PT198H
  mosg__grid_domain:            global
  mosg__grid_type:              standard
  mosg__grid_version:           1.6.0
  mosg__model_configuration:    gl_ens
  source:                       Met Office Unified Model
  title:                        MOGREPS-G Model Forecast on Global 20 km St...
  um_version:                   11.5
  Conventions:                  CF-1.7
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Dask code running on this machine has read the metadata for the file via NFS, but has not yet read in the data arrays themselves.&lt;/p&gt;
&lt;p&gt;Likewise we can see the observations, so we can perform a calculation locally. Here we average the predictions over the realisations and then compare them with the observations at a particular height. (This is a deliberately inefficient calculation, as we could average at only the required height, but you get the point.)&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="o"&gt;%%&lt;/span&gt;&lt;span class="n"&gt;time&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;scope&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/ecmwf/000490262cdd067721a34112963bcaa2b44860ab.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;observations&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/eumetsat/ad-hoc/observations.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="n"&gt;averages&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;predictions&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="s1"&gt;&amp;#39;realization&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;averages&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;observations&lt;/span&gt;
    &lt;span class="n"&gt;diff&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;#scope()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;CPU times: user 10 µs, sys: 2 µs, total: 12 µs
Wall time: 13.8 µs
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;When we uncomment &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scope()&lt;/span&gt;&lt;/code&gt; and actually run this, it takes over 14 minutes to complete! Accessing the data over NFS between data centres (we run this notebook in AWS) is just too slow.&lt;/p&gt;
&lt;p&gt;In fact just copying the data files onto the computer running this notebook takes the same sort of time. At least 2.8 GiB + 4.8 MiB of data must pass from the data centres to this machine to perform the calculation.&lt;/p&gt;
&lt;p&gt;Instead we should obviously run the Dask tasks where the data is. We can do that on a Dask cluster.&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/07/19/dask-multi-cloud.md&lt;/span&gt;, line 226)&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="running-up-a-cluster"&gt;
&lt;h1&gt;Running Up a Cluster&lt;/h1&gt;
&lt;p&gt;The cluster is run up with a single command. It takes a while though&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;subprocess&lt;/span&gt;

&lt;span class="n"&gt;scheduler_process&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;subprocess&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;Popen&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;
        &lt;span class="s1"&gt;&amp;#39;../dask_multicloud/dask-boot.sh&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="sa"&gt;f&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;rcar@&lt;/span&gt;&lt;span class="si"&gt;{&lt;/span&gt;&lt;span class="n"&gt;ecmwf_host&lt;/span&gt;&lt;span class="si"&gt;}&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="sa"&gt;f&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;rcar@&lt;/span&gt;&lt;span class="si"&gt;{&lt;/span&gt;&lt;span class="n"&gt;eumetsat_host&lt;/span&gt;&lt;span class="si"&gt;}&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&lt;/span&gt;
    &lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;[#] ip link add dasklocal type wireguard
[#] wg setconf dasklocal /dev/fd/63
[#] ip -6 address add fda5:c0ff:eeee:0::1/64 dev dasklocal
[#] ip link set mtu 1420 up dev dasklocal
[#] ip -6 route add fda5:c0ff:eeee:2::/64 dev dasklocal
[#] ip -6 route add fda5:c0ff:eeee:1::/64 dev dasklocal
2022-06-29 14:46:57,237 - distributed.scheduler - INFO - -----------------------------------------------
2022-06-29 14:46:58,602 - distributed.http.proxy - INFO - To route to workers diagnostics web server please install jupyter-server-proxy: python -m pip install jupyter-server-proxy
2022-06-29 14:46:58,643 - distributed.scheduler - INFO - -----------------------------------------------
2022-06-29 14:46:58,644 - distributed.scheduler - INFO - Clear task state
2022-06-29 14:46:58,646 - distributed.scheduler - INFO -   Scheduler at:     tcp://172.17.0.2:8786
2022-06-29 14:46:58,646 - distributed.scheduler - INFO -   dashboard at:                     :8787
2022-06-29 14:47:16,104 - distributed.scheduler - INFO - Register worker &amp;lt;WorkerState &amp;#39;tcp://[fda5:c0ff:eeee:1::11]:37977&amp;#39;, name: ecmwf-1-2, status: undefined, memory: 0, processing: 0&amp;gt;
2022-06-29 14:47:16,107 - distributed.scheduler - INFO - Starting worker compute stream, tcp://[fda5:c0ff:eeee:1::11]:37977
2022-06-29 14:47:16,108 - distributed.core - INFO - Starting established connection
2022-06-29 14:47:16,108 - distributed.scheduler - INFO - Register worker &amp;lt;WorkerState &amp;#39;tcp://[fda5:c0ff:eeee:1::11]:44575&amp;#39;, name: ecmwf-1-3, status: undefined, memory: 0, processing: 0&amp;gt;
2022-06-29 14:47:16,109 - distributed.scheduler - INFO - Starting worker compute stream, tcp://[fda5:c0ff:eeee:1::11]:44575
2022-06-29 14:47:16,109 - distributed.core - INFO - Starting established connection
2022-06-29 14:47:16,113 - distributed.scheduler - INFO - Register worker &amp;lt;WorkerState &amp;#39;tcp://[fda5:c0ff:eeee:1::11]:40121&amp;#39;, name: ecmwf-1-1, status: undefined, memory: 0, processing: 0&amp;gt;
2022-06-29 14:47:16,114 - distributed.scheduler - INFO - Starting worker compute stream, tcp://[fda5:c0ff:eeee:1::11]:40121
2022-06-29 14:47:16,114 - distributed.core - INFO - Starting established connection
2022-06-29 14:47:16,119 - distributed.scheduler - INFO - Register worker &amp;lt;WorkerState &amp;#39;tcp://[fda5:c0ff:eeee:1::11]:40989&amp;#39;, name: ecmwf-1-0, status: undefined, memory: 0, processing: 0&amp;gt;
2022-06-29 14:47:16,121 - distributed.scheduler - INFO - Starting worker compute stream, tcp://[fda5:c0ff:eeee:1::11]:40989
2022-06-29 14:47:16,121 - distributed.core - INFO - Starting established connection
2022-06-29 14:47:23,342 - distributed.scheduler - INFO - Register worker &amp;lt;WorkerState &amp;#39;tcp://[fda5:c0ff:eeee:2::11]:33423&amp;#39;, name: eumetsat-2-0, status: undefined, memory: 0, processing: 0&amp;gt;
2022-06-29 14:47:23,343 - distributed.scheduler - INFO - Starting worker compute stream, tcp://[fda5:c0ff:eeee:2::11]:33423
2022-06-29 14:47:23,343 - distributed.core - INFO - Starting established connection
2022-06-29 14:47:23,346 - distributed.scheduler - INFO - Register worker &amp;lt;WorkerState &amp;#39;tcp://[fda5:c0ff:eeee:2::11]:43953&amp;#39;, name: eumetsat-2-1, status: undefined, memory: 0, processing: 0&amp;gt;
2022-06-29 14:47:23,348 - distributed.scheduler - INFO - Starting worker compute stream, tcp://[fda5:c0ff:eeee:2::11]:43953
2022-06-29 14:47:23,348 - distributed.core - INFO - Starting established connection
2022-06-29 14:47:23,350 - distributed.scheduler - INFO - Register worker &amp;lt;WorkerState &amp;#39;tcp://[fda5:c0ff:eeee:2::11]:46089&amp;#39;, name: eumetsat-2-3, status: undefined, memory: 0, processing: 0&amp;gt;
2022-06-29 14:47:23,352 - distributed.scheduler - INFO - Starting worker compute stream, tcp://[fda5:c0ff:eeee:2::11]:46089
2022-06-29 14:47:23,352 - distributed.core - INFO - Starting established connection
2022-06-29 14:47:23,357 - distributed.scheduler - INFO - Register worker &amp;lt;WorkerState &amp;#39;tcp://[fda5:c0ff:eeee:2::11]:43727&amp;#39;, name: eumetsat-2-2, status: undefined, memory: 0, processing: 0&amp;gt;
2022-06-29 14:47:23,358 - distributed.scheduler - INFO - Starting worker compute stream, tcp://[fda5:c0ff:eeee:2::11]:43727
2022-06-29 14:47:23,358 - distributed.core - INFO - Starting established connection
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We need to wait for 8 &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed.core&lt;/span&gt; &lt;span class="pre"&gt;-&lt;/span&gt; &lt;span class="pre"&gt;INFO&lt;/span&gt; &lt;span class="pre"&gt;-&lt;/span&gt; &lt;span class="pre"&gt;Starting&lt;/span&gt; &lt;span class="pre"&gt;established&lt;/span&gt; &lt;span class="pre"&gt;connection&lt;/span&gt;&lt;/code&gt; lines - one from each of 4 worker processes on each of 2 worker machines.&lt;/p&gt;
&lt;p&gt;What has happened here is:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;start-scheduler.sh&lt;/span&gt;&lt;/code&gt; runs up a Docker container on this computer.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The container creates a WireGuard IPv6 data plane VPN. This involves generating shared keys for all the nodes and a network interface inside itself. This data plane VPN is transient and unique to this cluster.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The container runs a Dask scheduler, hosted on the data plane network.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It then asks each data centre to provision workers and routing.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Each data centre hosts a control process, accessible over the control plane network. On invocation:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;The control process creates a WireGuard network interface on the data plane network. This acts as a router between the workers inside the data centres and the scheduler.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It starts Docker containers on compute instances. These containers have their own WireGuard network interface on the data plane network, routing via the control process instance.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Docker containers spawn (4) Dask worker processes, each of which connects via the data plane network back to the scheduler created at the beginning.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The result is one container on this computer running the scheduler, talking to a container on each worker machine, over a throw-away data plane WireGuard IPv6 network which allows each of the (in this case 8) Dask worker processes to communicate with each other and the scheduler, even though they are partitioned over 3 data centres.&lt;/p&gt;
&lt;p&gt;Something like this&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;Image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;filename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;images/datacentres-dask.png&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;&lt;img alt="png" src="https://blog.dask.org/_images/dask-multi-cloud_35_0.png" /&gt;&lt;/p&gt;
&lt;p&gt;Key&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;span style='color: blue'&gt;Data plane network&lt;/span&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;span style='color: green'&gt;Dask&lt;/span&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;span style='color: red'&gt;NetCDF data&lt;/span&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 308)&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="connecting-to-the-cluster"&gt;
&lt;h1&gt;Connecting to the Cluster&lt;/h1&gt;
&lt;p&gt;The scheduler for the cluster is now running in a Docker container on this machine and is exposed on &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;localhost&lt;/span&gt;&lt;/code&gt;, so we can create a client talking to it&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;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;localhost:8786&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;2022-06-29 14:47:35,535 - distributed.scheduler - INFO - Receive client connection: Client-69f22f41-f7ba-11ec-a0a2-0acd18a5c05a
2022-06-29 14:47:35,536 - distributed.core - INFO - Starting established connection
/home/ec2-user/miniconda3/envs/jupyter/lib/python3.10/site-packages/distributed/client.py:1287: VersionMismatchWarning: Mismatched versions found

+---------+--------+-----------+---------+
| Package | client | scheduler | workers |
+---------+--------+-----------+---------+
| msgpack | 1.0.4  | 1.0.3     | 1.0.3   |
| numpy   | 1.23.0 | 1.22.3    | 1.22.3  |
| pandas  | 1.4.3  | 1.4.2     | 1.4.2   |
+---------+--------+-----------+---------+
Notes:
-  msgpack: Variation is ok, as long as everything is above 0.6
  warnings.warn(version_module.VersionMismatchWarning(msg[0][&amp;quot;warning&amp;quot;]))
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If you click through the client you should see the workers under the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Scheduler&lt;/span&gt; &lt;span class="pre"&gt;Info&lt;/span&gt;&lt;/code&gt; node&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="c1"&gt;# client&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can also click through to the Dashboard on http://localhost:8787/status. There we can show the workers on the task stream&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;show_all_workers&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="n"&gt;my_org&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;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ecmwf-1-0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;my_org&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;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ecmwf-1-1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;my_org&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;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ecmwf-1-2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;my_org&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;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ecmwf-1-3&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;my_org&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;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;eumetsat-2-0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;my_org&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;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;eumetsat-2-1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;my_org&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;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;eumetsat-2-2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;my_org&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;workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;eumetsat-2-3&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;0.5&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;show_all_workers&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/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 354)&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="running-on-the-cluster"&gt;
&lt;h1&gt;Running on the Cluster&lt;/h1&gt;
&lt;p&gt;Now that there is a Dask client in scope, calculations will be run on the cluster. We can define the tasks to be run&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;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/ecmwf/000490262cdd067721a34112963bcaa2b44860ab.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;observations&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/eumetsat/ad-hoc/observations.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;averages&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;predictions&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="s1"&gt;&amp;#39;realization&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;averages&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;observations&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;But when we try to perform the calculation it fails&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;with&lt;/span&gt; &lt;span class="n"&gt;pytest&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;raises&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="ne"&gt;FileNotFoundError&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;excinfo&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;show_all_workers&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;diff&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="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;excinfo&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;value&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&amp;quot;[Errno 2] No such file or directory: b&amp;#39;/data/eumetsat/ad-hoc/observations.nc&amp;#39;&amp;quot;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;It fails because the Dask scheduler has sent some of the tasks to read the data to workers running in EUMETSAT. They cannot see the data in ECMWF, and nor do we want them too, because reading all that data between data centres would be too slow.&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/07/19/dask-multi-cloud.md&lt;/span&gt;, line 380)&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="data-proximate-computation"&gt;
&lt;h1&gt;Data-Proximate Computation&lt;/h1&gt;
&lt;p&gt;Dask has the concept of &lt;a class="reference external" href="https://distributed.dask.org/en/stable/resources.html"&gt;resources&lt;/a&gt;. Tasks can be scheduled to run only where a resource (such as a GPU or amount of RAM) is available. We can &lt;a class="reference external" href="https://dask.discourse.group/t/understanding-work-stealing/335/13"&gt;abuse this mechanism&lt;/a&gt; to pin tasks to a data centre, by treating the data centre as a resource.&lt;/p&gt;
&lt;p&gt;To do this, when we create the workers we mark them as having a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pool-ecmwf&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pool-eumetsat&lt;/span&gt;&lt;/code&gt; resource. Then when we want to create tasks that can only run in one data centre, we annotate them as requiring the appropriate resource&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;with&lt;/span&gt; &lt;span class="p"&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;annotate&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;resources&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;pool-ecmwf&amp;#39;&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="n"&gt;predictions&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="s1"&gt;&amp;#39;realization&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;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We can hide that boilerplate inside a Python context manager - &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pool&lt;/span&gt;&lt;/code&gt; - and write&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;with&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ecmwf&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;predictions&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="s1"&gt;&amp;#39;realization&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;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pool&lt;/span&gt;&lt;/code&gt; context manager is a collaboration with the Dask developers, and is &lt;a class="reference external" href="https://github.com/gjoseph92/dask-worker-pools"&gt;published on GitHub&lt;/a&gt;. You can read more on the evolution of the concept on the &lt;a class="reference external" href="https://dask.discourse.group/t/understanding-work-stealing/335"&gt;Dask Discourse&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;We can do better than annotating the computation tasks though. If we load the data inside the context manager block, the data loading tasks will carry the annotation with them&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;with&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ecmwf&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/ecmwf/000490262cdd067721a34112963bcaa2b44860ab.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In this case we need another context manager in the library, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;propagate_pools&lt;/span&gt;&lt;/code&gt;, to ensure that the annotation is not lost when the task graph is processed and executed&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;with&lt;/span&gt; &lt;span class="n"&gt;propagate_pools&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="n"&gt;predictions&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="s1"&gt;&amp;#39;realization&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;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The two context managers allow us to annotate data with its pool, and hence where the loading tasks will run&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;with&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ecmwf&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/ecmwf/000490262cdd067721a34112963bcaa2b44860ab.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;eumetsat&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;observations&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/eumetsat/ad-hoc/observations.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;define some deferred calculations oblivious to the data’s provenance&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;averaged_predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;predictions&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="s1"&gt;&amp;#39;realization&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;averaged_predictions&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;observations&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;and then perform the final calculation&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="o"&gt;%%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;propagate_pools&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="n"&gt;show_all_workers&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;diff&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;CPU times: user 127 ms, sys: 6.34 ms, total: 133 ms
Wall time: 4.88 s
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Remember, our aim was to distribute a calculation across data centres, whilst preventing workers reading foreign bulk data.&lt;/p&gt;
&lt;p&gt;Here we know that data is only being read by workers in the appropriate location, because neither data centre can read the other’s data. Once data is in memory, Dask prefers to schedule tasks on the workers that have it, so that the local workers will tend to perform follow-on calcuations, and data chunks will tend to stay in the data centre that they were read from.&lt;/p&gt;
&lt;p&gt;Ordinarily though, if workers are idle, Dask would use them to perform calculations even if they don’t have the data. If allowed, this work stealing would result in data being moved between data centres unnecessarly, a potentially expensive operation. To prevent this, the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;propagate_pools&lt;/span&gt;&lt;/code&gt; context manager installs a scheduler optimisation that disallows work-stealing between workers in different pools.&lt;/p&gt;
&lt;p&gt;Once data loaded in one pool needs to be combined with data from another (the substraction in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;averaged_predictions.isel(height=10)&lt;/span&gt; &lt;span class="pre"&gt;-&lt;/span&gt; &lt;span class="pre"&gt;observations&lt;/span&gt;&lt;/code&gt; above), this is no longer classified as work stealing, and Dask will move data between data centres as required.&lt;/p&gt;
&lt;p&gt;That calculation in one go looks like this&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="o"&gt;%%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt;
&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ecmwf&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/ecmwf/000490262cdd067721a34112963bcaa2b44860ab.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;eumetsat&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="n"&gt;observations&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/eumetsat/ad-hoc/observations.nc&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;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;averages&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;predictions&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="s1"&gt;&amp;#39;realization&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;averages&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;observations&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;propagate_pools&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="n"&gt;show_all_workers&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;figure&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;figsize&lt;/span&gt;&lt;span class="o"&gt;=&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="mi"&gt;6&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
    &lt;span class="n"&gt;plt&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;diff&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;to_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="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="n"&gt;origin&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;lower&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;CPU times: user 234 ms, sys: 27.6 ms, total: 261 ms
Wall time: 6.04 s
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;img alt="png" src="https://blog.dask.org/_images/dask-multi-cloud_65_1.png" /&gt;&lt;/p&gt;
&lt;p&gt;In terms of code, compared with the local version above, this has only added the use of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;with&lt;/span&gt;&lt;/code&gt; blocks to label data and manage execution, and executes some 100 times faster.&lt;/p&gt;
&lt;p&gt;This is a best case, because in the demonstrator the files are actually hosted on the worker machines, so the speed difference between reading the files locally and reading them over NFS is maximized. Perhaps more persuasive is the measured volume of network traffic.&lt;/p&gt;
&lt;div class="pst-scrollable-table-container"&gt;&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr class="row-odd"&gt;&lt;th class="head text-left"&gt;&lt;p&gt;Method&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-center"&gt;&lt;p&gt;Time Taken&lt;/p&gt;&lt;/th&gt;
&lt;th class="head text-center"&gt;&lt;p&gt;Measured Network Traffic&lt;/p&gt;&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr class="row-even"&gt;&lt;td class="text-left"&gt;&lt;p&gt;Calculation over NFS&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;&amp;gt; 14 minutes&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;2.8 GiB&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td class="text-left"&gt;&lt;p&gt;Distributed Calculation&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;~ 10 seconds&lt;/p&gt;&lt;/td&gt;
&lt;td class="text-center"&gt;&lt;p&gt;8 MiB&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;/div&gt;
&lt;p&gt;Apart from some control and status messages, only the data required to paint the picture is sent over the network to this computer.&lt;/p&gt;
&lt;p&gt;Looking at the task stream we see the ECMWF workers (on the bottom) doing the bulk of the reading and computation, with the red transfer tasks joining this with data on EUMETSAT.&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;Image&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;filename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;images/task-graph.png&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;&lt;img alt="png" src="https://blog.dask.org/_images/dask-multi-cloud_67_0.png" /&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 494)&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="catalogs"&gt;
&lt;h1&gt;Catalogs&lt;/h1&gt;
&lt;p&gt;We can simplify this code even more. Because the data-loading tasks are labelled with their resource pool, this can be opaque to the scientist. So we can write&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;load_from_catalog&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;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;split&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)[&lt;/span&gt;&lt;span class="mi"&gt;2&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;xarray&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;open_dataset&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;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;chunk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;auto&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;allowing us to ignore where the data came from&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;predictions&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;load_from_catalog&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/ecmwf/000490262cdd067721a34112963bcaa2b44860ab.nc&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;observations&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;load_from_catalog&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/data/eumetsat/ad-hoc/observations.nc&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;averages&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;predictions&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="s1"&gt;&amp;#39;realization&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;averages&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;isel&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;height&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;observations&lt;/span&gt;

&lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;propagate_pools&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
    &lt;span class="n"&gt;show_all_workers&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
    &lt;span class="n"&gt;diff&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Of course the cluster would have to be provisioned with compute resources in the appropriate data centres, although with some work this could be made dynamic as part of the catalog code.&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/07/19/dask-multi-cloud.md&lt;/span&gt;, line 520)&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="more-information"&gt;
&lt;h1&gt;More Information&lt;/h1&gt;
&lt;p&gt;This &lt;a class="reference external" href="https://github.com/dmcg/dask-multicloud-poc/blob/main/demo/dask-multi-cloud.ipynb"&gt;notebook&lt;/a&gt;, and the code behind it, are published in a &lt;a class="reference external" href="https://github.com/dmcg/dask-multicloud-poc"&gt;GitHub repository&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;For details of the prototype implementation, and ideas for enhancements, see
&lt;a class="reference external" href="https://github.com/dmcg/dask-multicloud-poc/blob/main/demo/dask-multi-cloud-details.ipynb"&gt;dask-multi-cloud-details.ipynb&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/2022/07/19/dask-multi-cloud.md&lt;/span&gt;, line 527)&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="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;Thank you to Armagan Karatosun (EUMETSAT) and Vasileios Baousis (ECMWF) for their help and support with the infrastructure to support this proof of concept.
Gabe Joseph (Coiled) wrote the clever pool context managers, and Jacob Tomlinson (NVIDIA) reviewed this document.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Crown Copyright 2022&lt;/em&gt;&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2022/07/19/dask-multi-cloud/"/>
    <summary>This work is a joint venture between the Met Office and the European Weather Cloud, which is a partnership of ECMWF and EUMETSAT.</summary>
    <category term="deployment" label="deployment"/>
    <category term="distributed" label="distributed"/>
    <published>2022-07-19T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2021/03/11/dask_memory_usage/</id>
    <title>Measuring Dask memory usage with dask-memusage</title>
    <updated>2021-03-11T00:00:00+00:00</updated>
    <author>
      <name>&lt;a href="https://pythonspeed.com"&gt;Itamar Turner-Trauring&lt;/a&gt;</name>
    </author>
    <content type="html">&lt;p&gt;Using too much computing resources can get expensive when you’re scaling up in the cloud.&lt;/p&gt;
&lt;p&gt;To give a real example, I was working on the image processing pipeline for a spatial gene sequencing device, which could report not just which genes were being expressed but also where they were in a 3D volume of cells.
In order to get this information, a specialized microscope took snapshots of the cell culture or tissue, and the resulting data was run through a Dask pipeline.&lt;/p&gt;
&lt;p&gt;The pipeline was fairly slow, so I did some back-of-the-envelope math to figure out our computing costs would be once we started running more data for customers.
&lt;strong&gt;It turned out that we’d be using 70% of our revenue just paying for cloud computing!&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Clearly I needed to optimize this code.&lt;/p&gt;
&lt;p&gt;When we think about the bottlenecks in large-scale computation, we often focus on CPU: we want to use more CPU cores in order to get faster results.
Paying for all that CPU can be expensive, as in this case, and I did successfully reduce CPU usage by quite a lot.&lt;/p&gt;
&lt;p&gt;But high memory usage was also a problem, and fixing that problem led me to build a series of tools, tools that can also help you optimize and reduce your Dask memory usage.&lt;/p&gt;
&lt;p&gt;In the rest of this article you will learn:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="#problem"&gt;&lt;span class="xref myst"&gt;How high memory usage can drive up your computing costs&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How a tool called &lt;a class="reference external" href="https://github.com/itamarst/dask-memusage/"&gt;dask-memusage&lt;/a&gt; can help you &lt;a class="reference internal" href="#dask-memusage"&gt;&lt;span class="xref myst"&gt;find peak memory usage of the tasks in your Dask execution graph&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How to &lt;a class="reference internal" href="#fil"&gt;&lt;span class="xref myst"&gt;further pinpoint high memory usage&lt;/span&gt;&lt;/a&gt; using the &lt;a class="reference external" href="https://pythonspeed.com/fil"&gt;Fil memory profiler&lt;/a&gt;, so you can reduce memory usage.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/11/dask_memory_usage.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="the-problem-fixed-processing-chunks-and-a-high-memory-cpu-ratio-problem"&gt;

&lt;p&gt;As a reminder, I was working on a Dask pipeline that processed data from a specialized microscope.
The resulting data volume was quite large, and certain subsets of images had to be processed together as a unit.
From a computational standpoint, we effectively had a series of inputs X0, X1, X2, … that could be independently processed by a function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f()&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;The internal processing of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f()&lt;/span&gt;&lt;/code&gt; could not easily be parallelized further.
From a CPU scheduling perspective, this was fine, it was still an embarrassingly parallel problem given the large of number of X inputs.&lt;/p&gt;
&lt;p&gt;For example, if I provisioned a virtual machine with 4 CPU cores, to process the data I could start four processes, and each would max out a single core.
If I had 12 inputs and each processing step took about the same time, they might run as follows:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;CPU0: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X0)&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X4)&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X8)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;CPU1: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X1)&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X5)&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X9)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;CPU2: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X2)&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X6)&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X10)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;CPU3: &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X3)&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X7)&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(X11)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;If I could make &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f()&lt;/span&gt;&lt;/code&gt; faster, the pipeline as a whole would also run faster.&lt;/p&gt;
&lt;p&gt;CPU is not the only resource used in computation, however: RAM can also be a bottleneck.
For example, let’s say each call to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;f(Xi)&lt;/span&gt;&lt;/code&gt; took 12GB of RAM.
That means to fully utilize 4 CPUs, I would need 48GB of RAM—but what if my computer only has 16GB of RAM?&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Even though my computer has 4 CPUs, I can only utilize one CPU on a computer with 16GB RAM, because I don’t have enough RAM to run more than one task in parallel.&lt;/strong&gt;
In practice, these tasks ran in the cloud, where I could ensure the necessary RAM/core ratio was preserved by choosing the right pre-configured VM instances.
And on some clouds you can freely set the amount of RAM and number of CPU cores for each virtual machine you spin up.&lt;/p&gt;
&lt;p&gt;However, I didn’t quite know how much memory was used at peak, so I’d had to limit parallelism to reduce out-of-memory errors.
As a result, the default virtual machines we were using had half their CPUs resting idle, resources were paying for but not using.&lt;/p&gt;
&lt;p&gt;In order to provision hardware appropriately and max out all the CPUs, I needed to know how much peak memory each task was using.
And to do that, I created a new tool.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/11/dask_memory_usage.md&lt;/span&gt;, line 63)&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="measuring-peak-task-memory-usage-with-dask-memusage-dask-memusage"&gt;
&lt;h1&gt;Measuring peak task memory usage with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt; {#dask-memusage}&lt;/h1&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt; is a tool for measuring peak memory usage for each task in the Dask execution graph.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Per &lt;em&gt;task&lt;/em&gt; because Dask executes code as a graph of tasks, and the graph determines how much parallelism can be used.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Peak&lt;/em&gt; memory is important, because that is the bottleneck.
It doesn’t matter if average memory usage per task is 4GB, if two parallel tasks in the graph need 12GB each at the same time, you’re going to need 24GB of RAM if you want to to run both tasks on the same computer.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;section id="using-dask-memusage"&gt;
&lt;h2&gt;Using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt;&lt;/h2&gt;
&lt;p&gt;Since the gene sequencing code is proprietary and quite complex, let’s use a different example.
We’re going to count the occurrence of words in some text files, and then report the top-10 most common words in each file.
You can imagine combining the data later on, but we won’t bother with that in this simple 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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;sys&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;gc&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;time&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;sleep&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;pathlib&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;Path&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.bag&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;from_sequence&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;collections&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;Counter&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&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_memusage&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;calculate_top_10&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;file_path&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;span class="n"&gt;gc&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;collect&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# See notes below&lt;/span&gt;

    &lt;span class="c1"&gt;# Load the file&lt;/span&gt;
    &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="nb"&gt;open&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;file_path&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;data&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

    &lt;span class="c1"&gt;# Count the words&lt;/span&gt;
    &lt;span class="n"&gt;counts&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Counter&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;word&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;split&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;
        &lt;span class="n"&gt;counts&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;word&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;strip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;.,&amp;#39;&lt;/span&gt;&lt;span class="se"&gt;\&amp;quot;&lt;/span&gt;&lt;span class="s2"&gt;&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;lower&lt;/span&gt;&lt;span class="p"&gt;()]&lt;/span&gt; &lt;span class="o"&gt;+=&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;

    &lt;span class="c1"&gt;# Choose the top 10:&lt;/span&gt;
    &lt;span class="n"&gt;by_count&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;sorted&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;counts&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;items&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;key&lt;/span&gt;&lt;span class="o"&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="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
    &lt;span class="n"&gt;sleep&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;0.1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# See notes below&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;file_path&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;by_count&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;10&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;main&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;directory&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="c1"&gt;# Setup the calculation:&lt;/span&gt;

    &lt;span class="c1"&gt;# Create a 4-process cluster (running locally). Note only one thread&lt;/span&gt;
    &lt;span class="c1"&gt;# per-worker: because polling is per-process, you can&amp;#39;t run multiple&lt;/span&gt;
    &lt;span class="c1"&gt;# threads per worker, otherwise you&amp;#39;ll get results that combine memory&lt;/span&gt;
    &lt;span class="c1"&gt;# usage of multiple tasks.&lt;/span&gt;
    &lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;LocalCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;n_workers&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;threads_per_worker&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;memory_limit&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;# Install dask-memusage:&lt;/span&gt;
    &lt;span class="n"&gt;dask_memusage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;install&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;memusage.csv&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="c1"&gt;# Create the task graph:&lt;/span&gt;
    &lt;span class="n"&gt;files&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;Path&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;directory&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;iterdir&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
    &lt;span class="n"&gt;graph&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;files&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;calculate_top_10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;graph&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;span class="n"&gt;filename&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;example2.png&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;rankdir&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;TD&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

    &lt;span class="c1"&gt;# Run the calculations:&lt;/span&gt;
    &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;graph&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="nb"&gt;print&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="c1"&gt;# ... do something with results ...&lt;/span&gt;


&lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="vm"&gt;__name__&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;__main__&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
    &lt;span class="n"&gt;main&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;argv&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Here’s what the task graph looks like:&lt;/p&gt;
&lt;img src="/images/dask_memusage/example2.png" style="width: 75%; margin: 2em;"&gt;
&lt;p&gt;Plenty of parallelism!&lt;/p&gt;
&lt;p&gt;We can run the program on some files:&lt;/p&gt;
&lt;div class="highlight-shell-session notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;$ &lt;/span&gt;pip&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;dask&lt;span class="o"&gt;[&lt;/span&gt;bag&lt;span class="o"&gt;]&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;dask_memusage
&lt;span class="gp"&gt;$ &lt;/span&gt;python&lt;span class="w"&gt; &lt;/span&gt;example2.py&lt;span class="w"&gt; &lt;/span&gt;files/
&lt;span class="gp gp-VirtualEnv"&gt;(&amp;#39;frankenstein.txt&amp;#39;, [(&amp;#39;that&amp;#39;, 1016)&lt;/span&gt;&lt;span class="go"&gt;, (&amp;#39;was&amp;#39;, 1021), (&amp;#39;in&amp;#39;, 1180), (&amp;#39;a&amp;#39;, 1438), (&amp;#39;my&amp;#39;, 1751), (&amp;#39;to&amp;#39;, 2164), (&amp;#39;i&amp;#39;, 2754), (&amp;#39;of&amp;#39;, 2761), (&amp;#39;and&amp;#39;, 3025), (&amp;#39;the&amp;#39;, 4339)])&lt;/span&gt;
&lt;span class="gp gp-VirtualEnv"&gt;(&amp;#39;pride_and_prejudice.txt&amp;#39;, [(&amp;#39;she&amp;#39;, 1660)&lt;/span&gt;&lt;span class="go"&gt;, (&amp;#39;i&amp;#39;, 1730), (&amp;#39;was&amp;#39;, 1832), (&amp;#39;in&amp;#39;, 1904), (&amp;#39;a&amp;#39;, 1981), (&amp;#39;her&amp;#39;, 2142), (&amp;#39;and&amp;#39;, 3503), (&amp;#39;of&amp;#39;, 3705), (&amp;#39;to&amp;#39;, 4188), (&amp;#39;the&amp;#39;, 4492)])&lt;/span&gt;
&lt;span class="gp gp-VirtualEnv"&gt;(&amp;#39;greatgatsby.txt&amp;#39;, [(&amp;#39;that&amp;#39;, 564)&lt;/span&gt;&lt;span class="go"&gt;, (&amp;#39;was&amp;#39;, 760), (&amp;#39;he&amp;#39;, 770), (&amp;#39;in&amp;#39;, 849), (&amp;#39;i&amp;#39;, 999), (&amp;#39;to&amp;#39;, 1197), (&amp;#39;of&amp;#39;, 1224), (&amp;#39;a&amp;#39;, 1440), (&amp;#39;and&amp;#39;, 1565), (&amp;#39;the&amp;#39;, 2543)])&lt;/span&gt;
&lt;span class="gp gp-VirtualEnv"&gt;(&amp;#39;big.txt&amp;#39;, [(&amp;#39;his&amp;#39;, 40032)&lt;/span&gt;&lt;span class="go"&gt;, (&amp;#39;was&amp;#39;, 45356), (&amp;#39;that&amp;#39;, 47924), (&amp;#39;he&amp;#39;, 48276), (&amp;#39;a&amp;#39;, 83228), (&amp;#39;in&amp;#39;, 86832), (&amp;#39;to&amp;#39;, 114184), (&amp;#39;and&amp;#39;, 152284), (&amp;#39;of&amp;#39;, 159888), (&amp;#39;the&amp;#39;, 314908)])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;As one would expect, the most common words are stem words, but there is still some variation in order.&lt;/p&gt;
&lt;p&gt;Next, let’s look at the results from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-memusage-output-and-how-it-works"&gt;
&lt;h2&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt; output, and how it works&lt;/h2&gt;
&lt;p&gt;You’ll notice that the actual use of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt; involves just one extra line, other than the import:&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;dask_memusage&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;install&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;memusage.csv&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;What this will do is poll the process at 10ms intervals for peak memory usage, broken down by task.
In this case, here’s what &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;memusage.csv&lt;/span&gt;&lt;/code&gt; looks like:&lt;/p&gt;
&lt;div class="highlight-csv notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;task_key,min_memory_mb,max_memory_mb
&amp;quot;(&amp;#39;from_sequence-3637e6ff937ef8488894df60a80f62ed&amp;#39;, 3)&amp;quot;,51.2421875,51.2421875
&amp;quot;(&amp;#39;from_sequence-3637e6ff937ef8488894df60a80f62ed&amp;#39;, 0)&amp;quot;,51.70703125,51.70703125
&amp;quot;(&amp;#39;from_sequence-3637e6ff937ef8488894df60a80f62ed&amp;#39;, 1)&amp;quot;,51.28125,51.78515625
&amp;quot;(&amp;#39;from_sequence-3637e6ff937ef8488894df60a80f62ed&amp;#39;, 2)&amp;quot;,51.30859375,51.30859375
&amp;quot;(&amp;#39;calculate_top_10-afc867e38c3bd0aac8c18bb00d3634ca&amp;#39;, 2)&amp;quot;,56.19140625,56.19140625
&amp;quot;(&amp;#39;calculate_top_10-afc867e38c3bd0aac8c18bb00d3634ca&amp;#39;, 0)&amp;quot;,51.70703125,54.26953125
&amp;quot;(&amp;#39;calculate_top_10-afc867e38c3bd0aac8c18bb00d3634ca&amp;#39;, 1)&amp;quot;,52.30078125,52.30078125
&amp;quot;(&amp;#39;calculate_top_10-afc867e38c3bd0aac8c18bb00d3634ca&amp;#39;, 3)&amp;quot;,51.48046875,384.00390625
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For each task in the graph we are told minimum memory usage and peak memory usage, in MB.&lt;/p&gt;
&lt;p&gt;In more readable form:&lt;/p&gt;
&lt;div class="pst-scrollable-table-container"&gt;&lt;table class="table"&gt;
&lt;thead&gt;
&lt;tr class="row-odd"&gt;&lt;th class="head"&gt;&lt;p&gt;task_key&lt;/p&gt;&lt;/th&gt;
&lt;th class="head"&gt;&lt;p&gt;min_memory_mb&lt;/p&gt;&lt;/th&gt;
&lt;th class="head"&gt;&lt;p&gt;max_memory_mb&lt;/p&gt;&lt;/th&gt;
&lt;/tr&gt;
&lt;/thead&gt;
&lt;tbody&gt;
&lt;tr class="row-even"&gt;&lt;td&gt;&lt;p&gt;“(‘from_sequence-3637e6ff937ef8488894df60a80f62ed’, 3)”&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.2421875&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.2421875&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td&gt;&lt;p&gt;“(‘from_sequence-3637e6ff937ef8488894df60a80f62ed’, 0)”&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.70703125&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.70703125&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td&gt;&lt;p&gt;“(‘from_sequence-3637e6ff937ef8488894df60a80f62ed’, 1)”&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.28125&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.78515625&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td&gt;&lt;p&gt;“(‘from_sequence-3637e6ff937ef8488894df60a80f62ed’, 2)”&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.30859375&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.30859375&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td&gt;&lt;p&gt;“(‘calculate_top_10-afc867e38c3bd0aac8c18bb00d3634ca’, 2)”&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;56.19140625&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;56.19140625&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td&gt;&lt;p&gt;“(‘calculate_top_10-afc867e38c3bd0aac8c18bb00d3634ca’, 0)”&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.70703125&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;54.26953125&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-even"&gt;&lt;td&gt;&lt;p&gt;“(‘calculate_top_10-afc867e38c3bd0aac8c18bb00d3634ca’, 1)”&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;52.30078125&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;52.30078125&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;tr class="row-odd"&gt;&lt;td&gt;&lt;p&gt;“(‘calculate_top_10-afc867e38c3bd0aac8c18bb00d3634ca’, 3)”&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;51.48046875&lt;/p&gt;&lt;/td&gt;
&lt;td&gt;&lt;p&gt;384.00390625&lt;/p&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/tbody&gt;
&lt;/table&gt;
&lt;/div&gt;
&lt;p&gt;The bottom four lines are the interesting ones; all four start with a minimum memory usage of ~50MB RAM, and then memory may or may not increase as the code runs.
How much it increases presumably depends on the size of the files; most of them are quite small, so memory usage doesn’t change much.
&lt;strong&gt;One file uses much more maximum memory than the others, 384MB of RAM; presumably it’s &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;big.txt&lt;/span&gt;&lt;/code&gt; which is 25MB, since the other files are all smaller than 1MB.&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;The mechanism used, polling peak process memory, has some limitations:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;You’ll notice there’s a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;gc.collect()&lt;/span&gt;&lt;/code&gt; at the top of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;calculate_top_10()&lt;/span&gt;&lt;/code&gt;; this ensures we don’t count memory from previous code that hasn’t been cleaned up yet.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;There’s also a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;sleep()&lt;/span&gt;&lt;/code&gt; at the bottom of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;calculate_top_10()&lt;/span&gt;&lt;/code&gt;.
Because polling is used, tasks that run too quickly won’t get accurate information—the polling happens every 10ms or so, so you want to sleep at least 20ms.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Finally, because polling is per-process, you can’t run multiple threads per worker, otherwise you’ll get results that combine memory usage of multiple tasks.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
&lt;section id="interpreting-the-data"&gt;
&lt;h2&gt;Interpreting the data&lt;/h2&gt;
&lt;p&gt;What we’ve learned is that memory usage of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;calculate_top_10()&lt;/span&gt;&lt;/code&gt; grows with file size; this can be used to &lt;a class="reference external" href="https://pythonspeed.com/articles/estimating-memory-usage/"&gt;characterize the memory requirements for the workload&lt;/a&gt;.
That is, we can create a model that links data input sizes and required RAM, and then we can calculate the required RAM for any given level of parallelism.
And that can guide our choice of hardware, if we assume one task per CPU core.&lt;/p&gt;
&lt;p&gt;Going back to my original motivating problem, the gene sequencing pipeline: using the data from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt;, I was able to come up with a formula saying “for this size input, this much memory is necessary”.
Whenever we ran a batch job we could therefore set the parallelism as high as possible given the number of CPUs and RAM on the machine.&lt;/p&gt;
&lt;p&gt;While this allowed for more parallelism, it still wasn’t sufficient—processing was still using a huge amount of RAM, RAM that we had to pay for either with time (by using less CPUs) or money (by paying for more expensive virtual machines that more RAM).
So the next step was to reduce memory usage.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/11/dask_memory_usage.md&lt;/span&gt;, line 216)&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="reducing-memory-usage-with-fil-fil"&gt;
&lt;h1&gt;Reducing memory usage with Fil {#fil}&lt;/h1&gt;
&lt;p&gt;If we look at the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt; output for our word-counting example, the memory usage seems rather high: for a 25MB file, we’re using 330MB of RAM to count words.
Thinking through how an ideal version of this code might work, we ought to be able to process the file with much less memory (for example we could redesign our code to process the file line by line, reducing memory).&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;And that’s another way in which &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt; can be helpful: it can point us at specific code that needs memory usage optimized, at the granularity of a task.&lt;/strong&gt;
A task can be a rather large chunk of code, though, so the next step is to use a memory profiler that can point to specific lines of code.&lt;/p&gt;
&lt;p&gt;When working on the gene sequencing tool I used the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;memory_profiler&lt;/span&gt;&lt;/code&gt; package, and while that worked, and I managed to reduce memory usage quite a bit, I found it quite difficult to use.
It turns out that for batch data processing, the typical use case for Dask, &lt;a class="reference external" href="https://pythonspeed.com/articles/memory-profiler-data-scientists/"&gt;you want a different kind of memory profiler&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;So after I’d left that job, I created &lt;a class="reference external" href="https://pythonspeed.com/fil"&gt;a memory profiler called Fil&lt;/a&gt; that is expressly designed for finding peak memory usage.
Unlike &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt;, which can be run on production workloads, Fil slows down your execution and has other limitations I’m currently working on (it doesn’t support multiple processes, as of March 2021), so for now it’s better used for manual profiling.&lt;/p&gt;
&lt;p&gt;We can write a little script that only runs on &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;big.txt&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;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pathlib&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;Path&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;example2&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;calculate_top_10&lt;/span&gt;

&lt;span class="n"&gt;calculate_top_10&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;span class="s2"&gt;&amp;quot;files/big.txt&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;Run it under Fil:&lt;/p&gt;
&lt;div class="highlight-shell-session notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="go"&gt;pip install filprofiler&lt;/span&gt;
&lt;span class="go"&gt;fil-profile run example3.py&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And the result shows us where the bulk of the memory is being allocated:&lt;/p&gt;
&lt;iframe id="peak" src="/images/dask_memusage/peak-memory.svg" width="100%" height="300" scrolling="auto" frameborder="0"&gt;&lt;/iframe&gt;
&lt;p&gt;Reading in the file takes 8% of memory, but &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;data.split()&lt;/span&gt;&lt;/code&gt; is responsible for 84% of memory.
Perhaps we shouldn’t be loading the whole file into memory and splitting the whole file into words, and instead we should be processing the file line by line.
A good next step if this were real code would be to fix the way &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;calculate_top_10()&lt;/span&gt;&lt;/code&gt; is implemented.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2021/03/11/dask_memory_usage.md&lt;/span&gt;, line 254)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="next-steps"&gt;
&lt;h1&gt;Next steps&lt;/h1&gt;
&lt;p&gt;What should you do if your Dask workload is using too much memory?&lt;/p&gt;
&lt;p&gt;If you’re running Dask workloads with the Distributed backend, and you’re fine with only having one thread per worker, running with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt; will give you real-world per-task memory usage on production workloads.
You can then use the resulting information in a variety of ways:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;As a starting point for optimizing memory usage.
Once you know which tasks use the most memory, you can then &lt;a class="reference external" href="https://pythonspeed.com/articles/memory-profiler-data-scientists/"&gt;use Fil to figure out which lines of code are responsible&lt;/a&gt; and then use &lt;a class="reference external" href="https://pythonspeed.com/articles/data-doesnt-fit-in-memory/"&gt;a variety of techniques to reduce memory usage&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;When possible, you can fine tune your chunking size; smaller chunks will use less memory.
If you’re using Dask Arrays you can &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-chunks.html"&gt;set the chunk size&lt;/a&gt;; with Dask Dataframes you can &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-best-practices.html#repartition-to-reduce-overhead"&gt;ensure good partition sizes&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;You can fine tune your hardware configuration, so you’re not wasting RAM or CPU cores.
For example, on AWS you can &lt;a class="reference external" href="https://instances.vantage.sh/"&gt;choose a variety of instance sizes&lt;/a&gt; with different RAM/CPU ratios, one of which may match your workload characteristics.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;In my original use case, the gene sequencing pipeline, I was able to use a combination of lower memory use and lower CPU use to reduce costs to a much more modest level.
And when doing R&amp;amp;D, I was able to get faster results with the same hardware costs.&lt;/p&gt;
&lt;p&gt;You can &lt;a class="reference internal" href="#github.com/itamarst/dask-memusage/"&gt;&lt;span class="xref myst"&gt;learn more about &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-memusage&lt;/span&gt;&lt;/code&gt; here&lt;/span&gt;&lt;/a&gt;, and &lt;a class="reference external" href="https://pythonspeed.com/fil"&gt;learn more about the Fil memory profiler here&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2021/03/11/dask_memory_usage/"/>
    <summary>Using too much computing resources can get expensive when you’re scaling up in the cloud.</summary>
    <category term="dask" label="dask"/>
    <category term="distributed" label="distributed"/>
    <category term="memory" label="memory"/>
    <category term="profiling" label="profiling"/>
    <category term="ram" label="ram"/>
    <published>2021-03-11T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2020/07/30/beginners-config/</id>
    <title>Configuring a Distributed Dask Cluster</title>
    <updated>2020-07-30T00:00:00+00:00</updated>
    <author>
      <name>Julia Signell (Saturn Cloud)</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;Configuring a Dask cluster can seem daunting at first, but the good news is that the Dask project has a lot of built in heuristics that try its best to anticipate and adapt to your workload based on the machine it is deployed on and the work it receives. Possibly for a long time you can get away with not configuring anything special at all. That being said, if you are looking for some tips to move on from using Dask locally, or have a Dask cluster that you are ready to optimize with some more in-depth configuration, these tips and tricks will help guide you and link you to the best Dask docs on the topic!&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/2020/07/30/beginners-config.md&lt;/span&gt;, line 12)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="how-to-host-a-distributed-dask-cluster"&gt;

&lt;p&gt;The biggest jump for me was from running a local version of Dask for just an hour or so at a time during development, to standing up a production-ready version of Dask. Broadly there are two styles:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;a static dask cluster – one that is always on, always awake, always ready to accept work&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;an ephemeral dask cluster – one that is spun up or down easily with a Python API, and, when on, starts a minimal dask master node that itself only spins up dask workers when work is actually submitted&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Though those are the two broad main categories, there are tons of choices of how to actually achieve that. It depends on a number of factors including what cloud provider products you want to use and if those resources are pre-provisioned for you and whether you want to use a python API or a different deployment tool to actually start the Dask processes. A very exhaustive list of all the different ways you could provision a dask cluster is in the dask docs under &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup.html"&gt;Setup&lt;/a&gt;. As just a taste of what is described in those docs, you could:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Install and start up the dask processes &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/cli.html"&gt;manually from the CLI&lt;/a&gt; on cloud instances you provision, such as AWS EC2 or GCP GCE&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Use popular deployment interfaces such as &lt;a class="reference external" href="https://docs.dask.org/en/latest/setup/kubernetes-helm.html"&gt;helm for kubernetes&lt;/a&gt; to deploy dask in cloud container clusters you provision, such as AWS Fargate or GCP GKE&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Use ‘native’ deployment python APIs, provided by the dask developers, to create (and interactively configure) dask on deployment infrastructure they support, either through the general-purpose &lt;a class="reference external" href="https://gateway.dask.org/"&gt;Dask Gateway&lt;/a&gt; which supports multiple backends, or directly against cluster managers such as kubernetes with &lt;a class="reference external" href="https://kubernetes.dask.org/en/latest/"&gt;dask-kubernetes&lt;/a&gt; or YARN with &lt;a class="reference external" href="https://yarn.dask.org/en/latest/"&gt;dask-yarn&lt;/a&gt;, as long as you’ve already provisioned the kubernetes cluster or hadoop cluster already&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Use a nearly full-service deployment python API called &lt;a class="reference external" href="https://cloudprovider.dask.org/en/latest/"&gt;Dask Cloud Provider&lt;/a&gt;, that will go one step farther and provision the cluster for you too, as long as you give it AWS credentials (and as of time of writing, it only supports AWS)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;As you can see, there are a ton of options. On top of all of those, you might contract a managed service provider to provision and configure your dask cluster for you according to your specs, such as &lt;a class="reference external" href="https://www.saturncloud.io"&gt;Saturn Cloud&lt;/a&gt; (&lt;em&gt;Disclaimer: one of the authors (Julia Signell) works for Saturn Cloud&lt;/em&gt;).&lt;/p&gt;
&lt;p&gt;Whatever you choose, the whole point is to unlock the power of parallelism in Python that Dask provides, in as scalable a manner as possible which is what getting it running on distributed infrastructure is all about. Once you know where and with what API you are going to deploy your dask cluster, the real configuration process for your Dask cluster and its workload begins.&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/2020/07/30/beginners-config.md&lt;/span&gt;, line 30)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-to-choose-instance-type-for-your-cluster"&gt;
&lt;h1&gt;How to choose instance type for your cluster&lt;/h1&gt;
&lt;p&gt;When you are ready to set up your dask cluster for production, you will need to make some decisions about the infrastructure your scheduler and your workers will be running on, especially if you are using one of the options from &lt;a class="reference internal" href="#how-to-host-a-distributed-dask-cluster"&gt;&lt;span class="xref myst"&gt;How to host a distributed dask cluster&lt;/span&gt;&lt;/a&gt; that requires pre-provisioned infrastructure. Whether your infrastructure is on-prem or in the cloud, the classic decision points need to be made:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Memory requirements&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;CPU requirements&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Storage requirements&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;If you have tested your workload locally, a simple heuristic is to multiply the CPU, storage, and memory usage of your work by some multiplier that is related to how scaled down your local experiments are from your expected production usage. For example, if you test your workload locally with a 10% sample of data, multiplying any observed resource usage by at least 10 may get you close to your minimum instance size. Though in reality Dask’s many underlying optimizations means that it shouldn’t regularly require linear growth of resources to work on more data, this simple heuristic may give you a starting point as a good first pass technique.&lt;/p&gt;
&lt;p&gt;In the same vein, choosing the smallest instance and running with a predetermined subset of data and scaling up until it runs effectively gives you a hint towards the minimum instance size. If your local environment is too underpowered to run your flows locally with 10%+ of your source data, if it is a highly divergent environment (for example a different OS, or with many competing applications running in the background), or if it is difficult or annoying to monitor CPU, memory, and storage of your flow’s execution using your local machine, isolating the test case on the smallest workable node is a better option.&lt;/p&gt;
&lt;p&gt;On the flip side, choosing the biggest instance you can afford and observing the discrepancy between max CPU/memory/storage metrics and scaling back based on the ratio of unused resources can be a quicker way to find your ideal size.&lt;/p&gt;
&lt;p&gt;Wherever you land on node size might be heavily influenced by what you want to pay for, but as long as your node size is big enough that you are avoiding strict out of memory errors, the flip side of what you pay for with nodes closest to your minimum run specs is time. Since the point of your Dask cluster is to run distributed, parallel computations, you can get significant time savings if you scale up your instance to allow for more parallelism. If you have long running models that take hours to train that you can reduce to minutes, and get back some of your time or your employee’s time to see the feedback loop quickly, then scaling up over your minimum specs is worth it.&lt;/p&gt;
&lt;p&gt;Should your scheduler node and worker nodes be the same size? It may certainly be tempting to provision them at separate instance sizes to optimize resources. It’s worth a quick dive into the general resource requirements of each to get a good sense.&lt;/p&gt;
&lt;p&gt;For the scheduler, a serialized version of each task is submitted to it is held in memory for as long as it needs to determine which worker should take the work. This is not necessarily the same amount of memory needed to actually execute the task, but skimping too much on memory here may prevent work from being scheduled. From a CPU perspective, the needs of the scheduler are likely much lower than your workers, but starving the scheduler of CPU will cause deadlock, and when the scheduler is stuck or dies your workers also cannot get any work. Storage wise, the Dask scheduler does not persist much to disk, even temporarily, so it’s storage needs are quite low.&lt;/p&gt;
&lt;p&gt;For the workers, the specific resource needs of your task code may overtake any generalizations we can make. If nothing else, they need enough memory and CPU to deserialize each task payload, and serialize it up again to return as a Future to the Dask scheduler. Dask workers may persist the results of computations in memory, including distributed across the memory of the cluster, which you can read more about &lt;a class="reference external" href="https://distributed.dask.org/en/latest/memory.html"&gt;here&lt;/a&gt;. Regarding storage needs, fundamentally tasks submitted to Dask workers should not write to local storage - the scheduler does not guarantee work will be run on a given worker - so the storage costs should be directly related to the installation footprint of your worker’s dependencies and any ephemeral storage of the dask workers. Temporary files the workers create may include spilling in-memory data to local disk if they run out of memory as long as &lt;a class="reference external" href="https://docs.dask.org/en/latest/configuration-reference.html#distributed.worker.memory.spill"&gt;that behavior isn’t disabled&lt;/a&gt;, which means that reducing memory can have an effect on your ephemeral storage needs.&lt;/p&gt;
&lt;p&gt;Generally we would recommend simplifying your life and keeping your scheduler and worker nodes the same node size, but if you wanted to optimize them, use the above CPU, memory and storage patterns to give you a starting point for configuring them separately.&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/2020/07/30/beginners-config.md&lt;/span&gt;, line 54)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="how-to-choose-number-of-workers"&gt;
&lt;h1&gt;How to choose number of workers&lt;/h1&gt;
&lt;p&gt;Every dask cluster has one scheduler and any number of workers. The scheduler keeps track of what work needs to be done and what has already been completed. The workers do work, share results between themselves and report back to the scheduler. More background on what this entails is available in the &lt;a class="reference external" href="https://distributed.dask.org/en/latest/worker.html"&gt;dask.distributed documentation&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;When setting up a dask cluster you have to decide how many workers to use. It can be tempting to use many workers, but that isn’t always a good idea. If you use too many workers some may not have enough to do and spend much of their time idle. Even if they have enough to do, they might need to share data with each other which can be slow. Additionally if your machine has finite resources (rather than one node per worker), then each worker will be weaker - they might run out of memory, or take a long time to finish a task.&lt;/p&gt;
&lt;p&gt;On the other hand if you use too few workers you don’t get to take full advantage of the parallelism of dask and your work might take longer to complete overall.&lt;/p&gt;
&lt;p&gt;Before you decide how many workers to use, try using the default. In many cases dask can choose a default that makes use of the size and shape of your machine. If that doesn’t work, then you’ll need some information about the size and shape of your work. In particular you’ll want to know:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;What size is your computer or what types of compute nodes do you have access to?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How big is your data?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;What is the structure of the computation that you are trying to do?&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;If you are working on your local machine, then the size of the computer is fixed and knowable. If you are working on HPC or cloud instances then you can choose the resources allotted to each worker. You make the decision about the size of your cluster based on factors we discussed in &lt;a class="reference internal" href="#how-to-choose-instance-type-for-your-cluster"&gt;&lt;span class="xref myst"&gt;How to choose instance type for your cluster&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;Dask is often used in situations where the data are too big to fit in memory. In these cases the data are split into chunks or partitions. Each task is computed on the chunk and then the results are aggregated. You will learn about how to change the shape of your data &lt;a class="reference internal" href="#how-to-host-a-distributed-dask-cluster"&gt;&lt;span class="xref myst"&gt;below&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;The structure of the computation might be the hardest to reason about. If possible, it can be helpful to try out the computation on a very small subset of the data. You can see the task graph for a particular computation by calling &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.visualize()&lt;/span&gt;&lt;/code&gt;. If the graph is too large to comfortably view inline, then take a look at the &lt;a class="reference external" href="https://docs.dask.org/en/latest/diagnostics-distributed.html"&gt;Dask dashboard&lt;/a&gt; graph tab. This shows the task graph as it runs and lights up each section. To make dask most efficient, you want a task graph that isn’t too big or too interconnected. The &lt;a class="reference external" href="https://docs.dask.org/en/latest/best-practices.html#avoid-very-large-graphs"&gt;dask docs&lt;/a&gt; discuss several techniques for optimizing your task graph.&lt;/p&gt;
&lt;p&gt;To pick the number of workers to use, think about how many concurrent tasks are happening at any given part of the graph. If each task contains a non-trivial amount of work, then the fastest way to run dask is to have a worker for each concurrent task. For chunked data, if each worker is able to comfortably hold one data chunk in memory and do some computation on that data, then the number of chunks should be a multiple of the number of workers. This ensures that there is always enough work for a worker to do.&lt;/p&gt;
&lt;p&gt;If you have a highly variable number of tasks, then you can also consider using an adaptive cluster. In an adaptive cluster, you set the minimum and maximum number of workers and let the cluster add and remove workers as needed. When the scheduler determines that some workers aren’t needed anymore it asks the cluster to shut them down, and when more are needed, the scheduler asks the cluster to spin more up. This can work nicely for task graphs that start out with few input tasks then have more tasks in the middle, and then some aggregation or reductions at the end.&lt;/p&gt;
&lt;p&gt;Once you have started up your cluster with some workers, you can monitor their progress in the &lt;a class="reference external" href="https://docs.dask.org/en/latest/diagnostics-distributed.html"&gt;dask dashboard&lt;/a&gt;. There you can check on their memory consumption, watch their progress through the task graph, and access worker-level logs. Watching your computation in this way, provides insight into potential speedups and builds intuition about the number of workers to use in the future.&lt;/p&gt;
&lt;p&gt;The tricky bit about choosing the number of workers to use is that in practice the size and shape of your machine, data, and task graph can change. Figuring out how many workers to use can end up feeling like an endless fiddling of knobs. If this is starting to drive you crazy then remember that you can always change the number or workers, even while the cluster is running.&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/2020/07/30/beginners-config.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="how-to-choose-nthreads-to-utilize-multithreading"&gt;
&lt;h1&gt;How to choose nthreads to utilize multithreading&lt;/h1&gt;
&lt;p&gt;When starting dask workers themselves, there are two very important configuration options to play against each other: how many workers and how many threads per worker. You can actually manipulate both on the same worker process with flags, such as in the form &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask-worker&lt;/span&gt; &lt;span class="pre"&gt;--nprocs&lt;/span&gt; &lt;span class="pre"&gt;2&lt;/span&gt; &lt;span class="pre"&gt;--nthreads&lt;/span&gt; &lt;span class="pre"&gt;2&lt;/span&gt;&lt;/code&gt;, though &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--nprocs&lt;/span&gt;&lt;/code&gt; simply spins up another worker in the background so it is cleaner configuration to avoid setting &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--nprocs&lt;/span&gt;&lt;/code&gt; and instead manipulate that configuration with whatever you use to specify total number of workers. We already talked about &lt;a class="reference internal" href="#how-to-host-a-distributed-dask-cluster"&gt;&lt;span class="xref myst"&gt;how to choose number of workers&lt;/span&gt;&lt;/a&gt;, but you may modify your decision about that if you change a workers’ &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--nthreads&lt;/span&gt;&lt;/code&gt; to increase the amount of work an individual worker can do.&lt;/p&gt;
&lt;p&gt;When deciding the best number of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;nthreads&lt;/span&gt;&lt;/code&gt; for your workers, it all boils down to the type of work you expect those workers to do. The fundamental principle is that multiple threads are best to share data between tasks, but worse if running code that doesn’t release Python’s GIL (“Global Interpreter Lock”). Increasing the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;nthreads&lt;/span&gt;&lt;/code&gt; for work that does not release the Python’s GIL has no effect; the worker cannot use threading to optimize the speed of computation if the GIL is locked. This is a possible point of confusion for new Dask users who want to increase their parallelism, but don’t see any gains from increasing the threading limit of their workers.&lt;/p&gt;
&lt;p&gt;As discussed in &lt;a class="reference internal" href="#%5Bhttps://distributed.dask.org/en/latest/worker.html%5D(https://distributed.dask.org/en/latest/worker.html)"&gt;&lt;span class="xref myst"&gt;the Dask docs on workers&lt;/span&gt;&lt;/a&gt;, there are some rules of thumb when to worry about GIL lockages, and thus prefer more workers over heavier individual workers with high &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;nthreads&lt;/span&gt;&lt;/code&gt;:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;If your code is mostly pure Python (in non-optimized Python libraries) on non-numerical data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;If your code causes computations external to Python that are long running and don’t release the GIL explicitly&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Conveniently, a lot of dask users are running exclusively numerical computations using Python libraries optimized for multithreading, namely NumPy, Pandas, SciPy, etc in the PyData stack. If you do mostly numerical computations using those or similarly optimized libraries, you should emphasize a higher thread count. If you truly are doing mostly numerical computations, you can specify as many total threads as you have cores; if you are doing any work that would cause a thread to pause, for example any I/O (to write results to disk, perhaps), you can specify &lt;em&gt;more&lt;/em&gt; threads than you have cores, since some will be occasionally sitting idle. The ideal number regarding how many more threads than cores to set in that situation is complex to estimate and dependent on your workload, but taking some advice from &lt;a class="reference external" href="https://docs.python.org/3/library/concurrent.futures.html#concurrent.futures.ThreadPoolExecutor"&gt;concurrent.futures&lt;/a&gt;, 5 times the processors on your machine is a historical upper bound to limit your total thread count to for heavily I/O dependent workloads.&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/2020/07/30/beginners-config.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 id="how-to-chunk-arrays-and-partition-dataframes"&gt;
&lt;h1&gt;How to chunk arrays and partition DataFrames&lt;/h1&gt;
&lt;p&gt;There are many different methods of triggering work in dask. For instance: you can wrap functions with delayed or submit work directly to the client (for a comparison of the options see &lt;a class="reference external" href="https://docs.dask.org/en/latest/user-interfaces.html"&gt;User Interfaces&lt;/a&gt;). If you are loading structured data into dask objects, then you are likely using &lt;a class="reference external" href="https://docs.dask.org/en/latest/array.html"&gt;dask.array&lt;/a&gt; or &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe.html"&gt;dask.dataframe&lt;/a&gt;. These modules mimic numpy and pandas respectively - making it easier to interact with large arrays and large tabular datasets.&lt;/p&gt;
&lt;p&gt;When using dask.dataframe and dask.array, computations are divided among workers by splitting the data into pieces. In dask.dataframe these pieces are called &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-design.html#partitions"&gt;partitions&lt;/a&gt; and in dask.array they are called &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-chunks.html"&gt;chunks&lt;/a&gt;, but the principle is the same. In the case of dask.array each chunk holds a numpy array and in the case of dask.dataframe each partition holds a pandas dataframe. Either way, each one contains a small part of the data, but is representative of the whole and must be small enough to comfortably fit in worker memory.&lt;/p&gt;
&lt;p&gt;Often when loading in data, the partitions/chunks will be determined automatically. For instance, when reading from a directory containing many csv files, each file will become a partition. If your data are not split up by default, then it can be done manually using df.set_index or array.rechunk. If they are split up by default and you want to change the shape of the chunks, the file-level chunks should be a multiple of the dask level chunks (read more about this &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-best-practices.html#orient-your-chunks"&gt;here&lt;/a&gt;).&lt;/p&gt;
&lt;p&gt;As the user, you know how the data are going to be used, so you can often partition it in ways that lead to more efficient computations. For instance if you are going to be aggregating to a monthly step, it can make sense to chunk along the time axis. If instead you are going to be looking at a particular feature at different altitudes, it might make sense to chunk along the altitude. More tips for chunking dask.arrays are described in &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-best-practices.html"&gt;Best Practices&lt;/a&gt;. Another scenario in which it might be helpful to repartition is if you have filtered the data down to a subset of the original. In that case your partitions will likely be too small. See the dask.dataframe &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-best-practices.html#repartition-to-reduce-overhead"&gt;Best Practices&lt;/a&gt; for more details on how to handle that case.&lt;/p&gt;
&lt;p&gt;When choosing the size of chunks it is best to make them neither too small, nor too big (around 100MB is often reasonable). Each chunk needs to be able to fit into the worker memory and operations on that chunk should take some non-trivial amount of time (more than 100ms). For many more recommendations take a look at the docs on &lt;a class="reference external" href="https://docs.dask.org/en/latest/array-chunks.html"&gt;chunks&lt;/a&gt; and on &lt;a class="reference external" href="https://docs.dask.org/en/latest/dataframe-design.html#partitions"&gt;partitions&lt;/a&gt;.&lt;/p&gt;
&lt;hr class="docutils" /&gt;
&lt;p&gt;&lt;em&gt;We hope this helps you make decisions about whether to configure your Dask deployment differently and give you the confidence to try it out. We found all of this great information in the Dask docs, so if you are feeling inspired please follow the links we’ve sprinkled throughout and learn even more about Dask!&lt;/em&gt;&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2020/07/30/beginners-config/"/>
    <summary>Configuring a Dask cluster can seem daunting at first, but the good news is that the Dask project has a lot of built in heuristics that try its best to anticipate and adapt to your workload based on the machine it is deployed on and the work it receives. Possibly for a long time you can get away with not configuring anything special at all. That being said, if you are looking for some tips to move on from using Dask locally, or have a Dask cluster that you are ready to optimize with some more in-depth configuration, these tips and tricks will help guide you and link you to the best Dask docs on the topic!</summary>
    <category term="config" label="config"/>
    <category term="distributed" label="distributed"/>
    <published>2020-07-30T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2018/10/08/Dask-Jobqueue/</id>
    <title>Dask-jobqueue</title>
    <updated>2018-10-08T00:00:00+00:00</updated>
    <author>
      <name>Joe Hamman</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This work was done in collaboration with &lt;a class="reference external" href="https://github.com/mrocklin"&gt;Matthew Rocklin&lt;/a&gt; (Anaconda), Jim Edwards (NCAR), &lt;a class="reference external" href="https://github.com/guillaumeeb"&gt;Guillaume Eynard-Bontemps&lt;/a&gt; (CNES), and &lt;a class="reference external" href="https://github.com/lesteve"&gt;Loïc Estève&lt;/a&gt; (INRIA), and is supported, in part, by the US National Science Foundation &lt;a class="reference external" href="https://www.earthcube.org/"&gt;Earth Cube program&lt;/a&gt;. The dask-jobqueue package is a spinoff of the &lt;a class="reference external" href="https://medium.com/pangeo"&gt;Pangeo Project&lt;/a&gt;. This blogpost was previously published &lt;a class="reference external" href="https://medium.com/pangeo/dask-jobqueue-d7754e42ca53"&gt;here&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;TLDR;&lt;/strong&gt; &lt;em&gt;Dask-jobqueue&lt;/em&gt; allows you to seamlessly deploy &lt;a class="reference external" href="https://dask.org/"&gt;dask&lt;/a&gt; on HPC clusters that use a variety of job queuing systems such as PBS, Slurm, SGE, or LSF. Dask-jobqueue provides a &lt;em&gt;Pythonic&lt;/em&gt; user interface that manages dask workers/clusters through the submission, execution, and deletion of individual jobs on a HPC system. It gives users the ability to interactively scale workloads across large HPC systems; turning an interactive &lt;a class="reference external" href="http://jupyter.org/"&gt;Jupyter&lt;/a&gt; Notebook into a powerful tool for scalable computation on very large datasets.&lt;/p&gt;
&lt;p&gt;Install with:&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;conda&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;-c&lt;span class="w"&gt; &lt;/span&gt;conda-forge&lt;span class="w"&gt; &lt;/span&gt;dask-jobqueue
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;pip&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;dask-jobqueue
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And checkout the dask-jobqueue documentation: &lt;a class="reference external" href="http://jobqueue.dask.org"&gt;http://jobqueue.dask.org&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/2018/10/08/Dask-Jobqueue.md&lt;/span&gt;, line 28)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="introduction"&gt;

&lt;p&gt;Large high-performance computer (HPC) clusters are ubiquitous throughout the computational sciences. These HPC systems include powerful hardware, including many large compute nodes, high-speed interconnects, and parallel file systems. An example of such systems that we use at &lt;a class="reference external" href="https://ncar.ucar.edu/"&gt;NCAR&lt;/a&gt; is named &lt;a class="reference external" href="https://www2.cisl.ucar.edu/resources/computational-systems/cheyenne"&gt;Cheyenne&lt;/a&gt;. Cheyenne is a fairly large machine, with about 150k cores and over 300 TB of total memory.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Cheyenne is a 5.34-petaflops, high-performance computer operated by NCAR." src="https://cdn-images-1.medium.com/max/2000/1*Jqm612rTcdWFkmcZWhcrTw.jpeg" /&gt;&lt;em&gt;Cheyenne is a 5.34-petaflops, high-performance computer operated by NCAR.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;These systems frequently use a job queueing system, such as PBS, Slurm, or SGE, to manage the queueing and execution of many concurrent jobs from numerous users. A “job” is a single execution of a program that is to be run on some set of resources on the user’s HPC system. These jobs are often submitted via the command line:&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;qsub&lt;span class="w"&gt; &lt;/span&gt;do_thing_a.sh
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Where do_thing_a.sh is a shell script that might look something like this:&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="ch"&gt;#!/bin/bash&lt;/span&gt;
&lt;span class="c1"&gt;#PBS -N thing_a&lt;/span&gt;
&lt;span class="c1"&gt;#PBS -q premium&lt;/span&gt;
&lt;span class="c1"&gt;#PBS -A 123456789&lt;/span&gt;
&lt;span class="c1"&gt;#PBS -l select=1:ncpus=36:mem=109G&lt;/span&gt;

&lt;span class="nb"&gt;echo&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;“doing&lt;span class="w"&gt; &lt;/span&gt;thing&lt;span class="w"&gt; &lt;/span&gt;A”
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In this example “-N” specifies the name of this job, “-q” specifies the queue where the job should be run, “-A” specifies a project code to bill for the CPU time used while the job is run, and “-l” specifies the hardware specifications for this job. Each job queueing system has slightly different syntax for configuring and submitting these jobs.&lt;/p&gt;
&lt;p&gt;This interface has led to the development of a few common workflow patterns:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;MPI if you want to scale&lt;/em&gt;. MPI stands for the Message Passing Interface. It is a widely adopted interface allowing parallel computation across traditional HPC clusters. Many large computational models are written in languages like C and Fortran and use MPI to manage their parallel execution. For the old-timers out there, this is the go-to solution when it comes time to scale complex computations.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Batch it&lt;/em&gt;. It is quite common for scientific processing pipelines to include a few steps that can be easily parallelized by submitting multiple jobs in parallel. Maybe you want to “do_thing_a.sh” 500 times with slightly different inputs — easy, just submit all the jobs separately (or in what some queueing systems refer to as “array-job”).&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Serial is still okay&lt;/em&gt;. Computers are pretty fast these days, right? Maybe you don’t need to parallelize your programing at all. Okay, so keep it serial and get some coffee while your job is running.&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/2018/10/08/Dask-Jobqueue.md&lt;/span&gt;, line 62)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="the-problem"&gt;
&lt;h1&gt;The Problem&lt;/h1&gt;
&lt;p&gt;None of the workflow patterns listed above allow for interactive analysis on very large data analysis. When I’m prototyping new processing method, I often want to work interactively, say in a Jupyter Notebook. Writing MPI code on the fly is hard and expensive, batch jobs are inherently not interactive, and serial just won’t do when I start working on many TBs of data. Our experience is that these workflows tend to be fairly inelegant and difficult to transfer between applications, yielding lots of duplicated effort along the way.&lt;/p&gt;
&lt;p&gt;One of the aims of the Pangeo project is to facilitate interactive data on very large datasets. Pangeo leverages Jupyter and dask, along with a number of more domain specific packages like &lt;a class="reference external" href="http://xarray.pydata.org"&gt;xarray&lt;/a&gt; to make this possible. The problem is we didn’t have a particularly palatable method for deploying dask on our HPC clusters.&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/2018/10/08/Dask-Jobqueue.md&lt;/span&gt;, line 68)&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-system"&gt;
&lt;h1&gt;The System&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Jupyter Notebooks&lt;/em&gt; are web applications that support interactive code execution, display of figures and animations, and in-line explanatory text and equations. They are quickly becoming the standard open-source format for interactive computing in Python.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Dask&lt;/em&gt; is a library for parallel computing that coordinates well with Python’s existing scientific software ecosystem, including libraries like &lt;a class="reference external" href="http://www.numpy.org/"&gt;NumPy&lt;/a&gt;, &lt;a class="reference external" href="https://pandas.pydata.org/"&gt;Pandas&lt;/a&gt;, &lt;a class="reference external" href="http://scikit-learn.org/stable/"&gt;Scikit-Learn&lt;/a&gt;, and xarray. In many cases, it offers users the ability to take existing workflows and quickly scale them to much larger applications. &lt;a class="reference external" href="http://distributed.dask.org"&gt;*Dask-distributed&lt;/a&gt;* is an extension of dask that facilitates parallel execution across many computers.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;em&gt;Dask-jobqueue&lt;/em&gt; is a new Python package that we’ve built to facilitate the deployment of &lt;em&gt;dask&lt;/em&gt; on HPC clusters and interfacing with a number of job queuing systems. Its usage is concise and Pythonic.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&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_jobqueue&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;PBSCluster&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.distributed&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;

&lt;span class="n"&gt;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;PBSCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;36&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                     &lt;span class="n"&gt;memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;108GB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                     &lt;span class="n"&gt;queue&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;premium&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scale&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="n"&gt;client&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;section id="whats-happening-under-the-hood"&gt;
&lt;h2&gt;What’s happening under the hood?&lt;/h2&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;In the call to PBSCluster() we are telling dask-jobqueue how we want to configure each job. In this case, we set each job to have 1 &lt;em&gt;Worker&lt;/em&gt;, each using 36 cores (threads) and 108 GB of memory. We also tell the PBS queueing system we’d like to submit this job to the “premium” queue. This step also starts a Scheduler to manage workers that we’ll add later.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;It is not until we call the cluster.scale() method that we interact with the PBS system. Here we start 10 workers, or equivalently 10 PBS jobs. For each job, dask-jobqueue creates a shell command similar to the one above (except dask-worker is called instead of echo) and submits the job via a subprocess call.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Finally, we connect to the cluster by instantiating the Client class. From here, the rest of our code looks just as it would if we were using one of &lt;a class="reference external" href="http://docs.dask.org/en/stable/scheduler-overview.html"&gt;dask’s local schedulers&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Dask-jobqueue is easily customizable to help users capitalize on advanced HPC features. A more complicated example that would work on NCAR’s Cheyenne super computer is:&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;cluster&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;PBSCluster&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;36&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                    &lt;span class="n"&gt;processes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;18&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                    &lt;span class="n"&gt;memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;108GB&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                    &lt;span class="n"&gt;project&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;P48500028&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                    &lt;span class="n"&gt;queue&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;premium&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                    &lt;span class="n"&gt;resource_spec&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;select=1:ncpus=36:mem=109G&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                    &lt;span class="n"&gt;walltime&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;02:00:00&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                    &lt;span class="n"&gt;interface&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;ib0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                    &lt;span class="n"&gt;local_directory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;$TMPDIR&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In this example, we instruct the PBSCluster to 1) use up to 36 cores per job, 2) use 18 worker processes per job, 3) use the large memory nodes with 109 GB each, 4) use a longer walltime than is standard, 5) use the &lt;a class="reference external" href="https://en.wikipedia.org/wiki/InfiniBand"&gt;InfiniBand&lt;/a&gt; network interface (ib0), and 6) use the fast SSD disks as its local directory space.&lt;/p&gt;
&lt;p&gt;Finally, Dask offers the ability to “autoscale” clusters based on a set of heuristics. When the cluster needs more CPU or memory, it will scale up. When the cluster has unused resources, it will scale down. Dask-jobqueue supports this with a simple interface:&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;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;adapt&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;minimum&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;18&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;maximum&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;360&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In this example, we tell our cluster to autoscale between 18 and 360 workers (or 1 and 20 jobs).&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/2018/10/08/Dask-Jobqueue.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&gt;
&lt;section id="demonstration"&gt;
&lt;h1&gt;Demonstration&lt;/h1&gt;
&lt;p&gt;We have put together a fairly comprehensive screen cast that walks users through all the steps of setting up Jupyter and Dask (and dask-jobqueue) on an HPC cluster:&lt;/p&gt;
&lt;center&gt;&lt;iframe width="560" height="315" src="https://www.youtube.com/embed/FXsgmwpRExM" frameborder="0" allowfullscreen&gt;&lt;/iframe&gt;&lt;/center&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/2018/10/08/Dask-Jobqueue.md&lt;/span&gt;, line 125)&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="conclusions"&gt;
&lt;h1&gt;Conclusions&lt;/h1&gt;
&lt;p&gt;Dask jobqueue makes it much easier to deploy Dask on HPC clusters. The package provides a Pythonic interface to common job-queueing systems. It is also easily customizable.&lt;/p&gt;
&lt;p&gt;The autoscaling functionality allows for a fundamentally different way to do science on HPC clusters. Start your Jupyter Notebook, instantiate your dask cluster, and then do science — let dask determine when to scale up and down depending on the computational demand. We think this bursting approach to interactive parallel computing offers many benefits.&lt;/p&gt;
&lt;p&gt;Finally, in developing dask-jobqueue, we’ve run into a few challenges that are worth mentioning.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Queueing systems are highly customizable. System administrators seem to have a lot of control over their particularly implementation of each queueing system. In practice, this means that it is often difficult to simultaneously cover all permutations of a particular queueing system. We’ve generally found that things seem to be flexible enough and welcome feedback in the cases where they are not.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;CI testing has required a fair bit of work to setup. The target environment for using dask-jobqueue is on existing HPC clusters. In order to facilitate continuous integration testing of dask-jobqueue, we’ve had to configure multiple queueing systems (PBS, Slurm, SGE) to run in docker using Travis CI. This has been a laborious task and one we’re still working on.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We’ve built dask-jobqueue to operate in the dask-deploy framework. If you are familiar with &lt;a class="reference external" href="http://kubernetes.dask.org"&gt;dask-kubernetes&lt;/a&gt; or &lt;a class="reference external" href="http://yarn.dask.org"&gt;dask-yarn&lt;/a&gt;, you’ll recognize the basic syntax in dask-jobqueue as well. The coincident development of these dask deployment packages has recently brought up some important coordination discussions (e.g. &lt;a class="github reference external" href="https://github.com/dask/distributed/issues/2235"&gt;dask/distributed#2235&lt;/a&gt;).&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2018/10/08/Dask-Jobqueue/"/>
    <summary>This work was done in collaboration with Matthew Rocklin (Anaconda), Jim Edwards (NCAR), Guillaume Eynard-Bontemps (CNES), and Loïc Estève (INRIA), and is supported, in part, by the US National Science Foundation Earth Cube program. The dask-jobqueue package is a spinoff of the Pangeo Project. This blogpost was previously published here</summary>
    <category term="HPC" label="HPC"/>
    <category term="distributed" label="distributed"/>
    <category term="jobqueue" label="jobqueue"/>
    <published>2018-10-08T00:00:00+00:00</published>
  </entry>
</feed>
