<?xml version='1.0' encoding='UTF-8'?>
<feed xmlns="http://www.w3.org/2005/Atom" xml:lang="en">
  <id>https://blog.dask.org</id>
  <title>Dask Working Notes - Posts tagged IO</title>
  <updated>2026-03-05T15:05:23.204616+00:00</updated>
  <link href="https://blog.dask.org"/>
  <link href="https://blog.dask.org/blog/tag/io/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/07/23/extracting-fsspec-from-dask/</id>
    <title>Extracting fsspec from Dask</title>
    <updated>2019-07-23T00:00:00+00:00</updated>
    <author>
      <name>Martin Durant</name>
    </author>
    <content type="html">&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/07/23/extracting-fsspec-from-dask.md&lt;/span&gt;, line 9)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="tl-dr"&gt;

&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fsspec&lt;/span&gt;&lt;/code&gt;, the new base for file system operations in Dask, Intake, s3fs, gcsfs and others,
is now available as a stand-alone interface and central place to develop new backends
and file operations. Although it was developed as part of Dask, you no longer need Dask
to use this functionality.&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/07/23/extracting-fsspec-from-dask.md&lt;/span&gt;, line 16)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="introduction"&gt;
&lt;h1&gt;Introduction&lt;/h1&gt;
&lt;p&gt;Over the past few years, Dask’s IO capability has grown gradually and organically, to
include a number of file-formats, and the ability to access data seamlessly on various
remote/cloud data systems. This has been achieved through a number of sister packages
for viewing cloud resources as file systems, and dedicated code in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.bytes&lt;/span&gt;&lt;/code&gt;.
Some of the storage backends, particularly &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;s3fs&lt;/span&gt;&lt;/code&gt;, became immediately useful outside of
Dask too, and were picked up as optional dependencies by &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pandas&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;xarray&lt;/span&gt;&lt;/code&gt; and others.&lt;/p&gt;
&lt;p&gt;For the sake of consolidating the behaviours of the
various backends, providing a single reference specification for any new backends,
and to make this set of file system operations available even without Dask, I
created &lt;a class="reference external" href="https://filesystem-spec.readthedocs.io/en/latest/"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fsspec&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;.
This last week, Dask changed to use &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fsspec&lt;/span&gt;&lt;/code&gt; directly for its
IO needs, and I would like to describe in detail here the benefits of this change.&lt;/p&gt;
&lt;p&gt;Although this was done initially to easy the maintenance burden, the important takeaway
is that we want to make file systems operations easily available to the whole pydata ecosystem,
with or without 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/07/23/extracting-fsspec-from-dask.md&lt;/span&gt;, line 36)&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="history"&gt;
&lt;h1&gt;History&lt;/h1&gt;
&lt;p&gt;The first file system I wrote was &lt;a class="reference external" href="https://github.com/dask/hdfs3"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;hdfs3&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;, a thin wrapper
around the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;libhdfs3&lt;/span&gt;&lt;/code&gt; C library. At the time, Dask had acquired the ability to run on a
distributed cluster, and HDFS was the most popular storage solution for these (in the
commercial world, at least), so a solution was required. The python API closely matched
the C one, which in turn followed the Java API and posix standards. Fortunately, python already
has a &lt;a class="reference external" href="https://docs.python.org/3/library/io.html#i-o-base-classes"&gt;file-like standard&lt;/a&gt;, so
providing objects that implemented that was enough to make remote bytes available to many
packages.&lt;/p&gt;
&lt;p&gt;Pretty soon, it became apparent that cloud resources would be at least as important as in-cluster
file systems, and so followed &lt;a class="reference external" href="https://github.com/dask/s3fs"&gt;s3fs&lt;/a&gt;,
&lt;a class="reference external" href="https://github.com/Azure/azure-data-lake-store-python"&gt;adlfs&lt;/a&gt;, and &lt;a class="reference external" href="https://github.com/dask/gcsfs"&gt;gcsfs&lt;/a&gt;.
Each followed the same pattern, but with some specific code for the given interface, and
improvements based on the experience of the previous interfaces. During this time, Dask’s
needs also evolved, due to more complex file formats such as parquet. Code to interface to
the different backends and adapt their methods ended up in the Dask repository.&lt;/p&gt;
&lt;p&gt;In the meantime, other file system interfaces arrived, particularly
&lt;a class="reference external" href="https://arrow.apache.org/docs/python/filesystems.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pyarrow&lt;/span&gt;&lt;/code&gt;’s&lt;/a&gt;, which had its own HDFS
implementation and direct parquet reading. But we would like all of the tools in
the ecosystem to work together well, so that Dask can read parquet using either
engine from any of the storage backends.&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/07/23/extracting-fsspec-from-dask.md&lt;/span&gt;, line 61)&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="code-duplication"&gt;
&lt;h1&gt;Code duplication&lt;/h1&gt;
&lt;p&gt;Copying an interface, adapting it and releasing it, as I did with each iteration of the file system,
is certainly a quick way to get a job done. However, when you then want to change the behaviour, or
add new functionality, it turns out you need to repeat the work in each place
(violating the &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Don%27t_repeat_yourself"&gt;DRY&lt;/a&gt; principle) or have
the interfaces diverge slowly. Good examples of this were &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;glob&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;walk&lt;/span&gt;&lt;/code&gt;, which supported various
options for the former, and returned different things (list, versions dir/files iterator) for the
latter.&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;fs&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;bytes&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;local&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;LocalFileSystem&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;fs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;walk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/home/path/&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;&amp;lt;iterator of tuples&amp;gt;&lt;/span&gt;


&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;fs&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;s3fs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;S3FileSystme&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;fs&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;walk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;bucket/path&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;[list of filenames]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We found that, for Dask’s needs, we needed to build small wrapper
classes to ensure compatible APIs to all backends, as well as a class for operating on the local
file system with the same interface, and finally a registry for all of these with various helper
functions. Very little of this was specific to Dask, with only a couple of
functions concerning themselves with building graphs and deferred execution. It did, however,
raise the important issue that file systems should be serializable and that there should
be a way to specify a file to be opened, which is also serializable (and ideally supports
transparent text and compression).&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/07/23/extracting-fsspec-from-dask.md&lt;/span&gt;, line 91)&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="new-file-systems"&gt;
&lt;h1&gt;New file systems&lt;/h1&gt;
&lt;p&gt;I already mentioned the effort to make a local file system class which met the same interface as
the other ones which already existed. But there are more options that Dask users (and others)
might want, such as ssh, ftp, http, in-memory, and so on. Following requests from users to handle these options,
we started to write more file system interfaces, which all lived within &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.bytes&lt;/span&gt;&lt;/code&gt;; but it was unclear
whether they should only support very minimal functionality, just enough to get something done from
Dask, or a full set of file operations.&lt;/p&gt;
&lt;p&gt;The in-memory file system, in particular, existed in an extremely long-lived PR - it’s not
clear how useful such a thing is to Dask, when each worker has it’s own memory, and so sees
a different state of the “file system”.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/07/23/extracting-fsspec-from-dask.md&lt;/span&gt;, line 104)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="consolidation"&gt;
&lt;h1&gt;Consolidation&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/intake/filesystem_spec"&gt;file system Spec&lt;/a&gt;, later &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fsspec&lt;/span&gt;&lt;/code&gt;, was born out of a desire
to codify and consolidate the behaviours of the storage backends, reduce duplication, and provide the
same functionality to all backends. In the process, it became much easier to write new implementation
classes: see the &lt;a class="reference external" href="https://filesystem-spec.readthedocs.io/en/latest/api.html#built-in-implementations"&gt;implementation&lt;/a&gt;,
which include interesting and highly experimental options such as the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;CachingFileSystem&lt;/span&gt;&lt;/code&gt;, which
makes local copies of every remote read, for faster access the second time around. However, more
important main-stream implementations also took shape, such as FTP, SSH, Memory and webHDFS
(the latter being the best bet for accessing HDFS from outside the cluster, following all the
problems building and authenticating with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;hdfs3&lt;/span&gt;&lt;/code&gt;).&lt;/p&gt;
&lt;p&gt;Furthermore, the new repository gave the opportunity to implement new features, which would then have
further-reaching applicability than if they had been done in just selected repositories. Examples include
FUSE mounting, dictionary-style key-value views on file systems
(such as used by &lt;a class="reference external" href="https://zarr.readthedocs.io/en/stable/"&gt;zarr&lt;/a&gt;), and transactional writing of
files. All file systems are serializable and pyarrow-compliant.&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/07/23/extracting-fsspec-from-dask.md&lt;/span&gt;, line 122)&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="usefulness"&gt;
&lt;h1&gt;Usefulness&lt;/h1&gt;
&lt;p&gt;Eventually it dawned on my that the operations offered by the file system classes are very useful
for people not using Dask too. Indeed, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;s3fs&lt;/span&gt;&lt;/code&gt;, for example, sees plenty of use stand-alone, or in
conjunction with something like fastparquet, which can accept file system functions to its method,
or pandas.&lt;/p&gt;
&lt;p&gt;So it seemed to make sense to have a particular repo to write out the spec that a Dask-compliant
file system should adhere to, and I found that I could factor out a lot of common behaviour from
the existing implementations, provide functionality that had existed in only some to all, and
generally improve every implementation along the way.&lt;/p&gt;
&lt;p&gt;However, it was when considering &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fsspec&lt;/span&gt;&lt;/code&gt; in conjunction with &lt;a class="reference external" href="https://github.com/intake/intake/pull/381"&gt;Intake&lt;/a&gt;
that I realised how generally useful a stand-alone file system package can be: the PR
implemented a generalised file selector that can browse files in any file system that we
have available, even being able, for instance, to view a remote zip-file on S3 as a
browseable file system. Note that, similar to the general thrust of this blog, the
file selector itself need not live in the Intake repo and will eventually become either
its own thing, or an optional feature of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;fsspec&lt;/span&gt;&lt;/code&gt;. You shouldn’t need Intake either just
to get generalised file system operations.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2019/07/23/extracting-fsspec-from-dask.md&lt;/span&gt;, line 143)&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="final-thoughts"&gt;
&lt;h1&gt;Final Thoughts&lt;/h1&gt;
&lt;p&gt;This work is not quite on the level of “protocol standards” such as the well-know python buffer
protocol, but I think it is a useful step in making data in various storage services available
to people, since you can operate on each with the same API, expect the same behaviour, and
create real python file-like objects to pass to other functions. Having a single central repo
like this offers an obvious place to discuss and amend the spec, and build extra functionality
onto it.&lt;/p&gt;
&lt;p&gt;Many improvements remain to be done, such as support for globstrings in more functions, or
a single file system which can dispatch to the various backends depending on the form of the
URL provided; but there is now an obvious place for all of this to happen.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2019/07/23/extracting-fsspec-from-dask/"/>
    <summary>Document headings start at H2, not H1 [myst.header]</summary>
    <category term="IO" label="IO"/>
    <published>2019-07-23T00:00:00+00:00</published>
  </entry>
</feed>
