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

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

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

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

    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;config&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dataframe.backend&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;cudf&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cudf&lt;/span&gt;

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


&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;read_feather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;paths&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot;Create a Dask DataFrame from Feather files&lt;/span&gt;

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

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

&lt;span class="sd"&gt;    Returns&lt;/span&gt;
&lt;span class="sd"&gt;    -------&lt;/span&gt;
&lt;span class="sd"&gt;    dask.dataframe.DataFrame&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.dataframe&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dd&lt;/span&gt;
    &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pyarrow.dataset&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;ds&lt;/span&gt;

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

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

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

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

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

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

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

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

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

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

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

    &lt;span class="n"&gt;func&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;ReadFeather&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;columns&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;index&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2023/04/12/from-map.md&lt;/span&gt;, line 251)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="conclusion"&gt;
&lt;h1&gt;Conclusion&lt;/h1&gt;
&lt;p&gt;It is now easier than ever to create a Dask DataFrame collection from an arbitrary data source. Although the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.delayed&lt;/span&gt;&lt;/code&gt; API has already enabled similar functionality for many years, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt; now makes it possible to implement a custom IO function without sacrificing any of the high-level graph optimizations leveraged by the rest of the Dask DataFrame API.&lt;/p&gt;
&lt;p&gt;Start experimenting with &lt;a class="reference external" href="https://docs.dask.org/en/stable/generated/dask.dataframe.from_map.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;from_map&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; today, and let us know how it goes!&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2023/04/12/from-map/"/>
    <summary>Dask DataFrame provides dedicated IO functions for several popular tabular-data formats, like CSV and Parquet. If you are working with a supported format, then the corresponding function (e.g read_csv) is likely to be the most reliable way to create a new Dask DataFrame collection. For other workflows, from_map now offers a convenient way to define a DataFrame collection as an arbitrary function mapping. While these kinds of workflows have historically required users to adopt the Dask Delayed API, from_map now makes custom collection creation both easier and more performant.</summary>
    <category term="IO" label="IO"/>
    <category term="dataframe" label="dataframe"/>
    <published>2023-04-12T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2019/06/09/ucx-dgx/</id>
    <title>Experiments in High Performance Networking with UCX and DGX</title>
    <updated>2019-06-09T00:00:00+00:00</updated>
    <author>
      <name>Rick Zamora</name>
    </author>
    <content type="html">&lt;p&gt;&lt;em&gt;This post is about experimental and rapidly changing software.
Code examples in this post should not be relied upon to work in the future.&lt;/em&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/09/ucx-dgx.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="executive-summary"&gt;

&lt;p&gt;This post talks about connecting UCX, a high performance networking library, to
Dask, a parallel Python library, to accelerate communication-heavy workloads,
particularly when using GPUs.&lt;/p&gt;
&lt;p&gt;Additionally, we do this work on a DGX, a high-end multi-CPU multi-GPU machine
with a complex internal network. Working in this context was good to force
improvements in setting up Dask in heterogeneous situations targeting
different network cards, CPU sockets, GPUs, and so on..&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/09/ucx-dgx.md&lt;/span&gt;, line 23)&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="motivation"&gt;
&lt;h1&gt;Motivation&lt;/h1&gt;
&lt;p&gt;Many distributed computing workloads are communication-bound.
This is common in cases like the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Dataframe joins&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Machine learning algorithms&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Complex array computations&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Communication becomes a bigger bottleneck as we accelerate our computation,
such as when we use GPUs for computing.&lt;/p&gt;
&lt;p&gt;Historically, high performance communication was only available using MPI, or
with custom solutions. This post describes an effort to get close to the
communication bandwidth of MPI while still maintaining the ease of
programmability and accessibility of a dynamic system like Dask.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/09/ucx-dgx.md&lt;/span&gt;, line 40)&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="ucx-python-and-dask"&gt;
&lt;h1&gt;UCX, Python, and Dask&lt;/h1&gt;
&lt;p&gt;To get high performance networking in Dask, we wrapped UCX with Python and
then connected that to Dask.&lt;/p&gt;
&lt;p&gt;The &lt;a class="reference external" href="http://www.openucx.org/"&gt;OpenUCX&lt;/a&gt; project provides a uniform API around
various high performance networking libraries like InfiniBand, traditional
networking protocols like TCP/shared memory, and GPU-specific protocols like
NVLink. It is a layer beneath something like OpenMPI (the main user of OpenUCX
today) that figures out which networking system to use.&lt;/p&gt;
&lt;a href="http://www.openucx.org/wp-content/uploads/2015/07/ucx-architecture-1024x505.jpg"&gt;
&lt;img src="http://www.openucx.org/wp-content/uploads/2015/07/ucx-architecture-1024x505.jpg"
     width="100%" /&gt;&lt;/a&gt;
&lt;p&gt;Python users today don’t have much access to these network libraries, except
through MPI, which is sometimes not ideal. (&lt;a class="reference external" href="https://pypi.org/search/?q=infiniband"&gt;Try searching for “infiniband” on
PyPI.&lt;/a&gt;)&lt;/p&gt;
&lt;p&gt;This led us to create &lt;a class="reference external" href="https://github.com/rapidsai/ucx-py/"&gt;UCX-Py&lt;/a&gt;
.
UCX-Py is a Python wrapper around the UCX C library, which provides a Pythonic
API, both with blocking syntax appropriate for traditional HPC programs, as
well as a non-blocking &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;async/await&lt;/span&gt;&lt;/code&gt; syntax for more concurrent programs (like
Dask).
For more information on UCX I recommend watching Akshay’s &lt;a class="reference external" href="https://on-demand-gtc.gputechconf.com/gtcnew/sessionview.php?sessionName=s9679-ucx-python%3a+a+flexible+communication+library+for+python+applications"&gt;UCX
talk&lt;/a&gt;
from the GPU Technology Conference 2019.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Note: UCX-Py was primarily developed by &lt;a class="reference external" href="https://github.com/Akshay-Venkatesh/"&gt;Akshay Venkatesh&lt;/a&gt; (UCX, NVIDIA)
&lt;a class="reference external" href="https://tomaugspurger.github.io/"&gt;Tom Augspurger&lt;/a&gt; (Dask, Pandas, Anaconda),
and &lt;a class="reference external" href="https://github.com/quasiben/"&gt;Ben Zaitlen&lt;/a&gt; (NVIDIA, RAPIDS, Dask))&lt;/em&gt;&lt;/p&gt;
&lt;video width="560" height="315" controls&gt;
    &lt;source src="https://developer.download.nvidia.com/video/gputechconf/gtc/2019/video/S9679/s9679-ucx-python-a-flexible-communication-library-for-python-applications.mp4"
            type="video/mp4"&gt;
&lt;/video&gt;
&lt;p&gt;We then &lt;a class="reference external" href="https://github.com/dask/distributed/blob/master/distributed/comm/ucx.py"&gt;extended Dask communications to optionally use UCX&lt;/a&gt;.
If you have UCX and UCX-Py installed, then you can use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ucx://&lt;/span&gt;&lt;/code&gt; protocol in
addresses or the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;--protocol&lt;/span&gt; &lt;span class="pre"&gt;ucx&lt;/span&gt;&lt;/code&gt; flag when starting things up, something like
this.&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ dask-scheduler --protocol ucx
Scheduler started at ucx://127.0.0.1:8786

$ dask-worker ucx://127.0.0.1:8786
&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;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="gp"&gt;&amp;gt;&amp;gt;&amp;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="s1"&gt;&amp;#39;ucx://127.0.0.1:8786&amp;#39;&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/2019/06/09/ucx-dgx.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="experiment"&gt;
&lt;h1&gt;Experiment&lt;/h1&gt;
&lt;p&gt;We modified our &lt;a class="reference external" href="https://github.com/mrocklin/dask-gpu-benchmarks/blob/master/cupy-svd.ipynb"&gt;SVD with Dask and CuPy
benchmark&lt;/a&gt;
benchmark to use the UCX protocol for inter-process communication and ran it on
half of a DGX machine, using four GPUs. Here is a minimal implementation of the
UCX-enabled code:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cupy&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&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;wait&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_cuda&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;DGX&lt;/span&gt;

&lt;span class="c1"&gt;# Define DGX cluster and 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;DGX&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;span class="n"&gt;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 random data&lt;/span&gt;
&lt;span class="n"&gt;rs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;1000000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="c1"&gt;# Perform distributed SVD&lt;/span&gt;
&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;linalg&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;svd&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;persist&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;_&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;wait&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="n"&gt;u&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;By using UCX the overall communication times are reduced by an order of
magnitude. To produce the task-stream figures below, the benchmark was run on a
DGX-1 with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;CUDA_VISIBLE_DEVICES=[0,1,2,3]&lt;/span&gt;&lt;/code&gt;. It is clear that the red task
bars, corresponding to inter-process communication, are significantly
compressed. Communications that were taking 500ms-1s before now take around 20ms.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Before UCX&lt;/strong&gt;:&lt;/p&gt;
&lt;iframe src="https://matthewrocklin.com/raw-host/task_stream_lcc_dgx16.html" width="100%" height="200"&gt;&lt;/iframe&gt;
&lt;p&gt;&lt;strong&gt;After UCX&lt;/strong&gt;:&lt;/p&gt;
&lt;iframe src="https://matthewrocklin.com/raw-host/task_stream_dgx_dgx16.html" width="100%" height="200"&gt;&lt;/iframe&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/09/ucx-dgx.md&lt;/span&gt;, line 139)&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="diving-into-the-details"&gt;
&lt;h1&gt;Diving into the Details&lt;/h1&gt;
&lt;p&gt;On a GPU using NVLink we can get somewhere between 5-10 GB/s throughput between
pairs of GPUs. On a CPU this drops down to 1-2 GB/s (which seems well below
optimal).
These speeds can affect all Dask workloads (array, dataframe, xarray, ML, …),
but when the proper hardware is present, other bottlenecks may occur,
such as serialization when dealing with text or JSON-like data.&lt;/p&gt;
&lt;p&gt;This of course, depends on this fancy networking hardware being present.
On the GPU example above we’re mostly relying on NVLink, but we would also get
improved performance on an HPC InfiniBand network or even on a single laptop
machine using shared memory transports.&lt;/p&gt;
&lt;p&gt;The examples above was run on a DGX machine, which includes all of these
transports and more (as well as numerous GPUs).&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/09/ucx-dgx.md&lt;/span&gt;, line 156)&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="dgx"&gt;
&lt;h1&gt;DGX&lt;/h1&gt;
&lt;p&gt;The test machine used above was a
&lt;a class="reference external" href="https://www.nvidia.com/en-us/data-center/dgx-1/"&gt;DGX-1&lt;/a&gt;, which has eight GPUs,
two CPU sockets, four Infiniband network cards, and a complex NVLink
arrangement. This is a good example of non-uniform hardware. Certain CPUs
are closer to certain GPUs and network cards, and understanding this proximity
has an order-of-magnitude effect on performance. This situation isn’t unique
to DGX machines. The same situation arises when we have …&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Multiple workers in one node, with several nodes in a cluster&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multiple nodes in one rack, with several racks in a data center&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multiple data centers, such as is the case with hybrid cloud&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Working with the DGX was interesting because it forced us to start thinking
about heterogeneity, and making it easier to specify complex deployment scenarios
with Dask.&lt;/p&gt;
&lt;p&gt;Here is a diagram showing how the GPUs, CPUs, and Infiniband
cards are connected to each other in a DGX-1:&lt;/p&gt;
&lt;a href="https://docs.nvidia.com/dgx/bp-dgx/index.html#networking"&gt;
  &lt;img src="https://docs.nvidia.com/dgx/bp-dgx/graphics/networks.png"
         width="100%" /&gt;
&lt;/a&gt;
&lt;p&gt;And here the output of nvidia-smi showing the NVLink, networking, and CPU affinity
structure (this is mostly orthogonal to the structure displayed above).&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ nvidia-smi  topo -m
     GPU0  GPU1  GPU2  GPU3  GPU4  GPU5  GPU6  GPU7   ib0   ib1   ib2   ib3
GPU0   X    NV1   NV1   NV2   NV2   SYS   SYS   SYS   PIX   SYS   PHB   SYS
GPU1  NV1    X    NV2   NV1   SYS   NV2   SYS   SYS   PIX   SYS   PHB   SYS
GPU2  NV1   NV2    X    NV2   SYS   SYS   NV1   SYS   PHB   SYS   PIX   SYS
GPU3  NV2   NV1   NV2    X    SYS   SYS   SYS   NV1   PHB   SYS   PIX   SYS
GPU4  NV2   SYS   SYS   SYS    X    NV1   NV1   NV2   SYS   PIX   SYS   PHB
GPU5  SYS   NV2   SYS   SYS   NV1    X    NV2   NV1   SYS   PIX   SYS   PHB
GPU6  SYS   SYS   NV1   SYS   NV1   NV2    X    NV2   SYS   PHB   SYS   PIX
GPU7  SYS   SYS   SYS   NV1   NV2   NV1   NV2    X    SYS   PHB   SYS   PIX
ib0   PIX   PIX   PHB   PHB   SYS   SYS   SYS   SYS    X    SYS   PHB   SYS
ib1   SYS   SYS   SYS   SYS   PIX   PIX   PHB   PHB   SYS    X    SYS   PHB
ib2   PHB   PHB   PIX   PIX   SYS   SYS   SYS   SYS   PHB   SYS    X    SYS
ib3   SYS   SYS   SYS   SYS   PHB   PHB   PIX   PIX   SYS   PHB   SYS    X

    CPU Affinity
GPU0  0-19,40-59
GPU1  0-19,40-59
GPU2  0-19,40-59
GPU3  0-19,40-59
GPU4  20-39,60-79
GPU5  20-39,60-79
GPU6  20-39,60-79
GPU7  20-39,60-79

Legend:

  X    = Self
  SYS  = Traverse PCIe as well as the SMP interconnect between NUMA nodes
  NODE = Travrese PCIe as well as the interconnect between PCIe Host Bridges
  PHB  = Traverse PCIe as well as a PCIe Host Bridge (typically the CPU)
  PXB  = Traverse multiple PCIe switches (without PCIe Host Bridge)
  PIX  = Traverse a single PCIe switch
  NV#  = Traverse a bonded set of # NVLinks
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The DGX was originally designed for deep learning
applications. The complex network infrastructure above can be well used by
specialized NVIDIA networking libraries like
&lt;a class="reference external" href="https://developer.nvidia.com/nccl"&gt;NCCL&lt;/a&gt;, which knows how to route things
correctly, but is something of a challenge for other more general purpose
systems like Dask to adapt to.&lt;/p&gt;
&lt;p&gt;Fortunately, in meeting this challenge we were able to clean up a number of
related issues in Dask. In particular we can now:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Specify a more heterogeneous worker configuration when starting up a local cluster
&lt;a class="reference external" href="https://github.com/dask/distributed/pull/2675"&gt;dask/distributed #2675&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Learn bandwidth over time
&lt;a class="reference external" href="https://github.com/dask/distributed/pull/2658"&gt;dask/distributed #2658&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Add Worker plugins to help handle things like CPU affinity (though this is
quite general)
&lt;a class="reference external" href="https://github.com/dask/distributed/pull/2453"&gt;dask/distributed #2453&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;With these changes we’re now able to describe most of the DGX structure as
configuration in the Python function below:&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;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;Nanny&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;SpecCluster&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;Scheduler&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;distributed.worker&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;TOTAL_MEMORY&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_cuda.local_cuda_cluster&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;cuda_visible_devices&lt;/span&gt;


&lt;span class="k"&gt;class&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nc"&gt;CPUAffinity&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot; A Worker plugin to pin CPU affinity &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="fm"&gt;__init__&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cores&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;cores&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;setup&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="bp"&gt;self&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;worker&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
        &lt;span class="n"&gt;os&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sched_setaffinity&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="bp"&gt;self&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;cores&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;


&lt;span class="n"&gt;affinity&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;  &lt;span class="c1"&gt;# See nvidia-smi topo -m&lt;/span&gt;
    &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;60&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="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;60&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="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;79&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;79&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="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;79&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
    &lt;span class="mi"&gt;7&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;20&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;range&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;60&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;79&lt;/span&gt;&lt;span class="p"&gt;)),&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;DGX&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="s2"&gt;&amp;quot;ib&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;dashboard_address&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;:8787&amp;quot;&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;silence_logs&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
    &lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&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="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;
&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot; A Local Cluster for a DGX 1 machine&lt;/span&gt;

&lt;span class="sd"&gt;    NVIDIA&amp;#39;s DGX-1 machine has a complex architecture mapping CPUs,&lt;/span&gt;
&lt;span class="sd"&gt;    GPUs, and network hardware.  This function creates a local cluster&lt;/span&gt;
&lt;span class="sd"&gt;    that tries to respect this hardware as much as possible.&lt;/span&gt;

&lt;span class="sd"&gt;    It creates one Dask worker process per GPU, and assigns each worker&lt;/span&gt;
&lt;span class="sd"&gt;    process the correct CPU cores and Network interface cards to&lt;/span&gt;
&lt;span class="sd"&gt;    maximize performance.&lt;/span&gt;

&lt;span class="sd"&gt;    That being said, things aren&amp;#39;t perfect.  Today a DGX has very high&lt;/span&gt;
&lt;span class="sd"&gt;    performance between certain sets of GPUs and not others.  A Dask DGX&lt;/span&gt;
&lt;span class="sd"&gt;    cluster that uses only certain tightly coupled parts of the computer&lt;/span&gt;
&lt;span class="sd"&gt;    will have significantly higher bandwidth than a deployment on the&lt;/span&gt;
&lt;span class="sd"&gt;    entire thing.&lt;/span&gt;

&lt;span class="sd"&gt;    Parameters&lt;/span&gt;
&lt;span class="sd"&gt;    ----------&lt;/span&gt;
&lt;span class="sd"&gt;    interface: str&lt;/span&gt;
&lt;span class="sd"&gt;        The interface prefix for the infiniband networking cards.  This is&lt;/span&gt;
&lt;span class="sd"&gt;        often &amp;quot;ib&amp;quot;` or &amp;quot;bond&amp;quot;.  We will add the numeric suffix 0,1,2,3 as&lt;/span&gt;
&lt;span class="sd"&gt;        appropriate.  Defaults to &amp;quot;ib&amp;quot;.&lt;/span&gt;
&lt;span class="sd"&gt;    dashboard_address: str&lt;/span&gt;
&lt;span class="sd"&gt;        The address for the scheduler dashboard.  Defaults to &amp;quot;:8787&amp;quot;.&lt;/span&gt;
&lt;span class="sd"&gt;    CUDA_VISIBLE_DEVICES: str&lt;/span&gt;
&lt;span class="sd"&gt;        String like ``&amp;quot;0,1,2,3&amp;quot;`` or ``[0, 1, 2, 3]`` to restrict&lt;/span&gt;
&lt;span class="sd"&gt;        activity to different GPUs&lt;/span&gt;

&lt;span class="sd"&gt;    Examples&lt;/span&gt;
&lt;span class="sd"&gt;    --------&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;gt;&amp;gt;&amp;gt; from dask_cuda import DGX&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;gt;&amp;gt;&amp;gt; from dask.distributed import Client&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;gt;&amp;gt;&amp;gt; cluster = DGX(interface=&amp;#39;ib&amp;#39;)&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;gt;&amp;gt;&amp;gt; client = Client(cluster)&lt;/span&gt;
&lt;span class="sd"&gt;    &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt; &lt;span class="ow"&gt;is&lt;/span&gt; &lt;span class="kc"&gt;None&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;os&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;environ&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;CUDA_VISIBLE_DEVICES&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;0,1,2,3,4,5,6,7&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="nb"&gt;isinstance&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&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;CUDA_VISIBLE_DEVICES&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&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="s2"&gt;&amp;quot;,&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;int&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&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="n"&gt;TOTAL_MEMORY&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;8&lt;/span&gt;

    &lt;span class="n"&gt;spec&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
        &lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
            &lt;span class="s2"&gt;&amp;quot;cls&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;Nanny&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s2"&gt;&amp;quot;options&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;env&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
                    &lt;span class="s2"&gt;&amp;quot;CUDA_VISIBLE_DEVICES&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;cuda_visible_devices&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;
                        &lt;span class="n"&gt;ii&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt;
                    &lt;span class="p"&gt;),&lt;/span&gt;
                    &lt;span class="s2"&gt;&amp;quot;UCX_TLS&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;rc,cuda_copy,cuda_ipc&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                &lt;span class="p"&gt;},&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;interface&amp;quot;&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="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="o"&gt;//&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;protocol&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;ucx&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;ncores&amp;quot;&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="p"&gt;,&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;data&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;preload&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;dask_cuda.initialize_context&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;dashboard_address&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;:0&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;plugins&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;CPUAffinity&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;affinity&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;])],&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;silence_logs&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;silence_logs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                &lt;span class="s2"&gt;&amp;quot;memory_limit&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;memory_limit&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="p"&gt;},&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;ii&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="nb"&gt;enumerate&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
    &lt;span class="p"&gt;}&lt;/span&gt;

    &lt;span class="n"&gt;scheduler&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
        &lt;span class="s2"&gt;&amp;quot;cls&amp;quot;&lt;/span&gt;&lt;span class="p"&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;options&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;
            &lt;span class="s2"&gt;&amp;quot;interface&amp;quot;&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="nb"&gt;str&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;CUDA_VISIBLE_DEVICES&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="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
            &lt;span class="s2"&gt;&amp;quot;protocol&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;ucx&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
            &lt;span class="s2"&gt;&amp;quot;dashboard_address&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;dashboard_address&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="p"&gt;},&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;SpecCluster&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="n"&gt;spec&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="n"&gt;scheduler&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="n"&gt;silence_logs&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;silence_logs&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
        &lt;span class="o"&gt;**&lt;/span&gt;&lt;span class="n"&gt;kwargs&lt;/span&gt;
    &lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;However, we never got the NVLink structure down. The Dask scheduler currently
still assumes uniform bandwidths between workers. We’ve started to make small
steps towards changing this, but we’re not there yet (this will be useful as
well for people that want to think about in-rack or cross-data-center
deployments).&lt;/p&gt;
&lt;p&gt;As usual, in solving a highly specific problem, we were able to solve a number
of lingering general features, which then made our specific problem easy to
write down.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/09/ucx-dgx.md&lt;/span&gt;, line 373)&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="future-work"&gt;
&lt;h1&gt;Future Work&lt;/h1&gt;
&lt;p&gt;There has been significant effort over the last few months make everything
above work. In particular we …&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Modified UCX to support client-server workloads&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Wrapped UCX with UCX-Py and design a Python async-await friendly interface&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Wrapped UCX-Py with Dask&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Hooked everything together to make generic workloads function well&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The result is quite nice, especially for more communication heavy workloads.
However there is still plenty to do. This section details what we’re thinking
about now to continue this work.&lt;/p&gt;
&lt;ul&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Routing within complex networks&lt;/strong&gt;:
If you restrict yourself to four of the eight GPUs in a DGX, you can get 5-12 GB/s
between pairs of GPUs. For some workloads this can be significant. It
makes the system feel much more like a single unit than a bunch of isolated
machines.&lt;/p&gt;
&lt;p&gt;However we still can’t get great performance across the whole DGX because
there are many GPU-pairs that are not connected by NVLink, and so we get 10x
slower speeds. These dominate communication costs if you naively try to use
the full DGX.&lt;/p&gt;
&lt;p&gt;This might be solved either by:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Teaching Dask to avoid these communications&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Teaching UCX to route communications like these through a chain of
multiple NVLink connections&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Avoiding complex networks altogether. Newer systems like the DGX-2 use
NVSwitch, which provides uniform connectivity, with each GPU connected
to every other GPU.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;em&gt;Edit&lt;/em&gt;: I’ve since learned that UCX should be able to handle this. We
should still get PCIe speeds (around 4-7 GB/s) even when we don’t have
NVLink once an upstream bug gets fixed. Hooray!&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;CPU:&lt;/strong&gt; We can get 1-2 GB/s across InfiniBand, which isn’t bad, but also
wasn’t the full 5-8 GB/s that we were hoping for. This deserves more serious
profiling to determine what is going wrong. The current guess is that this
has to do with memory allocations.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt; &lt;span class="n"&gt;_&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="sa"&gt;b&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;0&amp;#39;&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;1000000000&lt;/span&gt;  &lt;span class="c1"&gt;# 1 GB&lt;/span&gt;
&lt;span class="n"&gt;CPU&lt;/span&gt; &lt;span class="n"&gt;times&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;user&lt;/span&gt; &lt;span class="mi"&gt;248&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;223&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;total&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;472&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;
&lt;span class="n"&gt;Wall&lt;/span&gt; &lt;span class="n"&gt;time&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;470&lt;/span&gt; &lt;span class="n"&gt;ms&lt;/span&gt;   &lt;span class="c1"&gt;# &amp;lt;&amp;lt;----- Around 2 GB/s.  Slower than I expected&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Probably we’re just doing something dumb here.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Package UCX:&lt;/strong&gt; Currently I’m building the UCX and UCX-Py libraries from
source (see appendix below for instructions). Ideally these would become
conda packages. &lt;a class="reference external" href="https://github.com/jakirkham"&gt;John Kirkham&lt;/a&gt; (Conda Forge,
NVIDIA, Dask) is taking a look at this along with the UCX developers from
Mellanox.&lt;/p&gt;
&lt;p&gt;See &lt;a class="reference external" href="https://github.com/rapidsai/ucx-py/issues/65"&gt;ucx-py #65&lt;/a&gt; for
more information.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Learn Heterogeneous Bandwidths:&lt;/strong&gt; In order to make good scheduling
decisions Dask needs to estimate how long it will take to move data between
machines. This question is now becoming much more complex, and depends on
both the source and destination machines (the network topology) the data
type (NumPy array, GPU array, Pandas Dataframe with text) and more. In
complex situations our bandwidths can span a 100x range (100 MB/s to 10
GB/s).&lt;/p&gt;
&lt;p&gt;Dask will have to develop more complex models for bandwidth, and
learn these over time.&lt;/p&gt;
&lt;p&gt;See &lt;a class="reference external" href="https://github.com/dask/distributed/issues/2743"&gt;dask/distributed
#2743&lt;/a&gt; for more
information.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Support other GPU libraries:&lt;/strong&gt; To send GPU data around we need to teach
Dask how to serialize Python objects into GPU buffers. There is code in
the dask/distributed repository to do this for Numba, CuPy, and RAPIDS cuDF
objects, but we’ve really only tested CuPy seriously. We should expand
this by some of the following steps:&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Try a distributed Dask cuDF join computation&lt;/p&gt;
&lt;p&gt;See &lt;a class="reference external" href="https://github.com/dask/distributed/pull/2746"&gt;dask/distributed #2746&lt;/a&gt; for initial work here.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Teach Dask to serialize array GPU libraries, like PyTorch and
TensorFlow, or possibly anything that supports the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__cuda_array_interface__&lt;/span&gt;&lt;/code&gt; protocol.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Track down communication failures:&lt;/strong&gt; We still occasionally get
unexplained communication failures. We should stress test this system to
discover rough corners.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;TCP&lt;/strong&gt;: Groups with high performing TCP networks can’t yet make use of UCX+Dask (though they can use either one individually).&lt;/p&gt;
&lt;p&gt;Currently using UCX in a client-server mode as we’re doing with
Dask requires access to RDMA libraries, which are often not found on systems
without networking systems like InfiniBand. This means that groups with
high performing TCP networks can’t make use of UCX+Dask.&lt;/p&gt;
&lt;p&gt;This is in progress at &lt;a class="reference external" href="https://github.com/openucx/ucx/pull/3570"&gt;openucx/ucx #3570&lt;/a&gt;&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Commodity Hardware&lt;/strong&gt;: Currently this code is only really useful on
high performance Linux systems that have InfiniBand or NVLink. However,
it would be nice to also use this on more commodity systems, including
personal laptop computers using TCP and shared memory.&lt;/p&gt;
&lt;p&gt;Currently Dask uses TCP for inter-process communication on a single machine.
Using UCX on a personal computer would give us access to shared memory
speeds, which tend to be an order of magnitude faster.&lt;/p&gt;
&lt;p&gt;See &lt;a class="reference external" href="https://github.com/openucx/ucx/issues/3663"&gt;openucx/ucx #3663&lt;/a&gt; for more
information.&lt;/p&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;strong&gt;Tune Performance:&lt;/strong&gt; The 5-10 GB/s bandwidths that we see with NVLink
today are sub-optimal. With UCX-Py alone we’re able to get something like
15 GB/s on large message tranfers. We should benchmark and tune our
implementation to see what is taking up the extra time. Until things work
more robustly though, this is a secondary priority.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/06/09/ucx-dgx.md&lt;/span&gt;, line 493)&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="appendix-setup"&gt;
&lt;h1&gt;Appendix: Setup&lt;/h1&gt;
&lt;p&gt;Performing these experiments depends currently on development branches in a few
repositories. This section includes my current setup.&lt;/p&gt;
&lt;section id="create-conda-environment"&gt;
&lt;h2&gt;Create Conda Environment&lt;/h2&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;conda&lt;/span&gt; &lt;span class="n"&gt;create&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;n&lt;/span&gt; &lt;span class="n"&gt;ucx&lt;/span&gt; &lt;span class="n"&gt;python&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;3.7&lt;/span&gt; &lt;span class="n"&gt;libtool&lt;/span&gt; &lt;span class="n"&gt;cmake&lt;/span&gt; &lt;span class="n"&gt;automake&lt;/span&gt; &lt;span class="n"&gt;autoconf&lt;/span&gt; &lt;span class="n"&gt;cython&lt;/span&gt; &lt;span class="n"&gt;bokeh&lt;/span&gt; &lt;span class="n"&gt;pytest&lt;/span&gt; &lt;span class="n"&gt;pkg&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;config&lt;/span&gt; &lt;span class="n"&gt;ipython&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt; &lt;span class="n"&gt;numba&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Note: for some reason using conda-forge makes the autogen step below fail.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="set-up-ucx"&gt;
&lt;h2&gt;Set up UCX&lt;/h2&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;# Clone UCX repository and get branch
git clone https://github.com/openucx/ucx
cd ucx
git remote add Akshay-Venkatesh git@github.com:Akshay-Venkatesh/ucx.git
git remote update Akshay-Venkatesh
git checkout ucx-cuda

# Build
git clean -xfd
export CUDA_HOME=/usr/local/cuda-9.2/
./autogen.sh
mkdir build
cd build
../configure --prefix=$CONDA_PREFIX --enable-debug --with-cuda=$CUDA_HOME --enable-mt --disable-cma CPPFLAGS=&amp;quot;-I//usr/local/cuda-9.2/include&amp;quot;
make -j install

# Verify
ucx_info -d
which ucx_info  # verify that this is in the conda environment

# Verify that we see NVLink speeds
ucx_perftest -t tag_bw -m cuda -s 1048576 -n 1000 &amp;amp; ucx_perftest dgx15 -t tag_bw -m cuda -s 1048576 -n 1000
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="set-up-ucx-py"&gt;
&lt;h2&gt;Set up UCX-Py&lt;/h2&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;git clone git@github.com:rapidsai/ucx-py
cd ucx-py

export UCX_PATH=$CONDA_PREFIX
make install
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="set-up-dask"&gt;
&lt;h2&gt;Set up Dask&lt;/h2&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;git&lt;/span&gt; &lt;span class="n"&gt;clone&lt;/span&gt; &lt;span class="n"&gt;git&lt;/span&gt;&lt;span class="nd"&gt;@github&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;com&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;dask&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;git&lt;/span&gt;
&lt;span class="n"&gt;cd&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;
&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;e&lt;/span&gt; &lt;span class="o"&gt;.&lt;/span&gt;
&lt;span class="n"&gt;cd&lt;/span&gt; &lt;span class="o"&gt;..&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;git&lt;/span&gt; &lt;span class="n"&gt;clone&lt;/span&gt; &lt;span class="n"&gt;git&lt;/span&gt;&lt;span class="nd"&gt;@github&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;com&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;distributed&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;git&lt;/span&gt;
&lt;span class="n"&gt;cd&lt;/span&gt; &lt;span class="n"&gt;distributed&lt;/span&gt;
&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;e&lt;/span&gt; &lt;span class="o"&gt;.&lt;/span&gt;
&lt;span class="n"&gt;cd&lt;/span&gt; &lt;span class="o"&gt;..&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="optionally-set-up-cupy"&gt;
&lt;h2&gt;Optionally set up cupy&lt;/h2&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;cuda92&lt;/span&gt;&lt;span class="o"&gt;==&lt;/span&gt;&lt;span class="mi"&gt;6&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="optionally-set-up-cudf"&gt;
&lt;h2&gt;Optionally set up cudf&lt;/h2&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;conda&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt; &lt;span class="n"&gt;rapidsai&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;nightly&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt; &lt;span class="n"&gt;conda&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;forge&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt; &lt;span class="n"&gt;numba&lt;/span&gt; &lt;span class="n"&gt;cudf&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;cudf&lt;/span&gt; &lt;span class="n"&gt;cudatoolkit&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;9.2&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="optionally-set-up-jupyterlab"&gt;
&lt;h2&gt;Optionally set up JupyterLab&lt;/h2&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;conda&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;ipykernel&lt;/span&gt; &lt;span class="n"&gt;jupyterlab&lt;/span&gt; &lt;span class="n"&gt;nb_conda_kernels&lt;/span&gt; &lt;span class="n"&gt;nodejs&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;For the Dask dashboard&lt;/p&gt;
&lt;div class="highlight-default notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;pip&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask_labextension&lt;/span&gt;
&lt;span class="n"&gt;jupyter&lt;/span&gt; &lt;span class="n"&gt;labextension&lt;/span&gt; &lt;span class="n"&gt;install&lt;/span&gt; &lt;span class="n"&gt;dask&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="n"&gt;labextension&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="my-benchmark"&gt;
&lt;h2&gt;My Benchmark&lt;/h2&gt;
&lt;p&gt;I’ve been using the following benchmark to test communication. It allocates a
chunked Dask array, and then adds it to its transpose, which forces a lot of
communication, but not much computation.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="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;defaultdict&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;asyncio&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;time&lt;/span&gt;
&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pprint&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;pprint&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;cupy&lt;/span&gt;

&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="kn"&gt;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;wait&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;distributed.utils&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;format_time&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;format_bytes&lt;/span&gt;

&lt;span class="k"&gt;async&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;f&lt;/span&gt;&lt;span class="p"&gt;():&lt;/span&gt;

    &lt;span class="c1"&gt;# Set up workers on the local machine&lt;/span&gt;
    &lt;span class="k"&gt;async&lt;/span&gt; &lt;span class="k"&gt;with&lt;/span&gt; &lt;span class="n"&gt;DGX&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;asynchronous&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;silence_logs&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;cluster&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
        &lt;span class="k"&gt;async&lt;/span&gt; &lt;span class="k"&gt;with&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="n"&gt;asynchronous&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="kc"&gt;True&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="k"&gt;as&lt;/span&gt; &lt;span class="n"&gt;client&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;

            &lt;span class="c1"&gt;# Create a simple random array&lt;/span&gt;
            &lt;span class="n"&gt;rs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cupy&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;RandomState&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;40000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;40000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;chunks&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;128 MiB&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;persist&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;npartitions&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;chunks&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="k"&gt;await&lt;/span&gt; &lt;span class="n"&gt;wait&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="c1"&gt;# Add X to its transpose, forcing computation&lt;/span&gt;
            &lt;span class="n"&gt;y&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;T&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
            &lt;span class="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&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;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;y&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

            &lt;span class="c1"&gt;# Collect, aggregate, and print peer-to-peer bandwidths&lt;/span&gt;
            &lt;span class="n"&gt;incoming_logs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;await&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;run&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;dask_worker&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;dask_worker&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;incoming_transfer_log&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
            &lt;span class="n"&gt;bandwidths&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;defaultdict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;list&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;k&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;L&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;incoming_logs&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="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;L&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                    &lt;span class="k"&gt;if&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;total&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;&amp;gt;&lt;/span&gt; &lt;span class="mi"&gt;1_000_000&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;
                        &lt;span class="n"&gt;bandwidths&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;k&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;who&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;append&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;bandwidth&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;])&lt;/span&gt;
            &lt;span class="n"&gt;bandwidths&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;w1&lt;/span&gt;&lt;span class="p"&gt;]&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;cluster&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;scheduler&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;workers&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;w2&lt;/span&gt;&lt;span class="p"&gt;]&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="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;format_bytes&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;+&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;/s&amp;#39;&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;quantile&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;v&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mf"&gt;0.25&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;0.50&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;0.75&lt;/span&gt;&lt;span class="p"&gt;])]&lt;/span&gt;
                &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;w1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;w2&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;bandwidths&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="p"&gt;}&lt;/span&gt;
            &lt;span class="n"&gt;pprint&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;bandwidths&lt;/span&gt;&lt;span class="p"&gt;)&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;asyncio&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get_event_loop&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;run_until_complete&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;em&gt;Note: most of this example is just getting back diagnostics, which can be
easily ignored. Also, you can drop the async/await code if you like. I think
that there should probably be more examples in the world using Dask with
async/await syntax, so I decided to leave it in.&lt;/em&gt;&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/06/09/ucx-dgx/"/>
    <summary>This post is about experimental and rapidly changing software.
Code examples in this post should not be relied upon to work in the future.</summary>
    <category term="python" label="python"/>
    <category term="scipy" label="scipy"/>
    <published>2019-06-09T00:00:00+00:00</published>
  </entry>
</feed>
