<?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 by Duncan McGregor (Met Office)</title>
  <updated>2026-03-05T15:05:19.510082+00:00</updated>
  <link href="https://blog.dask.org"/>
  <link href="https://blog.dask.org/blog/author/duncan-mcgregor-met-office/atom.xml" rel="self"/>
  <generator uri="https://ablog.readthedocs.io/" version="0.11.12">ABlog</generator>
  <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>
</feed>
