<?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 - Posted in 2015</title>
  <updated>2026-03-05T15:05:22.658740+00:00</updated>
  <link href="https://blog.dask.org"/>
  <link href="https://blog.dask.org/blog/2015/atom.xml" rel="self"/>
  <generator uri="https://ablog.readthedocs.io/" version="0.11.12">ABlog</generator>
  <entry>
    <id>https://blog.dask.org/2015/12/21/dask-year/</id>
    <title>Dask is one year old</title>
    <updated>2015-12-21T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr: Dask turned one yesterday. We discuss success and failures.&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;Dask began one year ago yesterday with the &lt;a class="reference external" href="https://github.com/blaze/dask/commit/05488db498c1561d266c7b676b8a89021c03a9e7"&gt;following
commit&lt;/a&gt;
(with slight edits here for clarity’s sake).&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;istask&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="k"&gt;return&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;x&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="nb"&gt;tuple&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="ow"&gt;and&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="ow"&gt;and&lt;/span&gt; &lt;span class="nb"&gt;callable&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&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;get&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="n"&gt;key&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;d&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;key&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;istask&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;func&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;args&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;v&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;v&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="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;func&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="o"&gt;*&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;get&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="n"&gt;arg&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;arg&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;args&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;v&lt;/span&gt;

 &lt;span class="o"&gt;...&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="ow"&gt;and&lt;/span&gt; &lt;span class="n"&gt;around&lt;/span&gt; &lt;span class="mi"&gt;50&lt;/span&gt; &lt;span class="n"&gt;lines&lt;/span&gt; &lt;span class="n"&gt;of&lt;/span&gt; &lt;span class="n"&gt;tests&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;this is a very inefficient scheduler&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Since then dask has matured, expanded to new domains, gathered &lt;a class="reference external" href="https://raw.githubusercontent.com/blaze/dask/master/AUTHORS.md"&gt;excellent
developers&lt;/a&gt;,
and spawned other open source projects. I thought it’d be a good time to look
back on what worked, what didn’t, and what we should work on in the future.&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/2015/12/21/dask-year.md&lt;/span&gt;, line 41)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="collections"&gt;

&lt;p&gt;Most users experience dask through the high-level collections of
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array/bag/dataframe/imperative&lt;/span&gt;&lt;/code&gt;. Each of these evolve as projects of
their own with different user groups and different levels of maturity.&lt;/p&gt;
&lt;section id="dask-array"&gt;
&lt;h2&gt;dask.array&lt;/h2&gt;
&lt;p&gt;The parallel larger-than-memory array module dask.array has seen the most
success of the dask components. It is the oldest, most mature, and most
sophisticated subproject. Much of dask.array’s use comes from downstream
projects, notably &lt;a class="reference external" href="http://xray.readthedocs.org/en/stable/"&gt;xray&lt;/a&gt; which seems to
have taken off in climate science. Dask.array also sees a fair amount of use
in imaging, genomics, and numerical algorithms research.&lt;/p&gt;
&lt;p&gt;People that I don’t know now use dask.array to do scientific research. From my
perspective that’s mission accomplished.&lt;/p&gt;
&lt;p&gt;There are still tweaks to make to algorithms, particularly as we scale out to
distributed systems (see far below).&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-bag"&gt;
&lt;h2&gt;dask.bag&lt;/h2&gt;
&lt;p&gt;Dask.bag started out as a weekend project and didn’t evolve much beyond that.
Fortunately there wasn’t much to do and this submodule probably has the highest
value/effort ratio .&lt;/p&gt;
&lt;p&gt;Bag doesn’t get as much attention as its older sibling array though. It’s
handy but not as well used and so not as robust.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-dataframe"&gt;
&lt;h2&gt;dask.dataframe&lt;/h2&gt;
&lt;p&gt;Dataframe is an interesting case, it’s both pretty sophisticated, pretty
mature, and yet also probably generates the most user frustration.&lt;/p&gt;
&lt;p&gt;Dask.dataframe gains a lot of value by leveraging Pandas both under the hood
(one dask DataFrame is many pandas DataFrames) and by copying its API (Pandas
users can use dask.dataframe without learning a new API.) However, because
dask.dataframe only implements a core subset of Pandas, users end up tripping
up on the missing functionality.&lt;/p&gt;
&lt;p&gt;This can be decomposed into to issues:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;It’s not clear that there exists a core subset of Pandas that would handle most
use cases. Users touch many diffuse parts of Pandas in a single workflow.
What one user considers core another user considers fringe. It’s not clear
how to agree on a sufficient subset to implement.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Once you implement this subset (and we’ve done our best) it’s hard to
convey expectations to the user about what is and is not available.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;That being said, dask.dataframe is pretty solid. It’s very fast, expressive,
and handles common use cases well. It probably generates the most
StackOverflow questions. This signals both confusion and active use.&lt;/p&gt;
&lt;p&gt;Special thanks here go out to Jeff Reback, for making Pandas release the GIL
and to Masaaki Horikoshi (&amp;#64;sinhrks) for greatly improving the maturity of
dask.dataframe.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="dask-imperative"&gt;
&lt;h2&gt;dask.imperative&lt;/h2&gt;
&lt;p&gt;Also known as &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.do&lt;/span&gt;&lt;/code&gt; this little backend remains one of the most powerful
and one of the least used (outside of myself.) We should rethink the API here
and improve learning materials.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="general-thoughts-on-collections"&gt;
&lt;h2&gt;General thoughts on collections&lt;/h2&gt;
&lt;p&gt;&lt;em&gt;Warning: this section is pretty subjective&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Big data collections are cool but perhaps less useful than people expect.
Parallel applications are often more complex than can be easily described by a
big array or a big dataframe. Many real-world parallel computations end up
being more particular in their parallelism needs. That’s not to say that the
array and dataframe abstractions aren’t central to parallel computing, it’s
just that we should not restrict ourselves to them. The world is more complex.&lt;/p&gt;
&lt;p&gt;However, it’s reasonable to break this “world is complex” rule within
particular domains. NDArrays seem to work well in climate science.
Specialized large dataframes like Dato’s SFrame seem to be effective for a
particular class of machine learning algorithms. The SQL table is inarguably
an effective abstraction in business intelligence. Large collections are
useful in specific contexts, but they are perhaps the focus of too much
attention. The big dataframe in particular is over-hyped.&lt;/p&gt;
&lt;p&gt;Most of the really novel and impressive work I’ve seen with dask has been done
either with custom graphs or with the dask.imperative API. I think we should
consider APIs that enable users to more easily express custom algorithms.&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/2015/12/21/dask-year.md&lt;/span&gt;, line 128)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="avoid-parallelism"&gt;
&lt;h1&gt;Avoid Parallelism&lt;/h1&gt;
&lt;p&gt;When giving talks on parallelism I’ve started to give a brief “avoid
parallelism” section. From the problems I see on stack overflow and from
general interactions when people run into performance challenges their first
solution seems to be to parallelize. This is sub-optimal. It’s often far
cheaper to improve storage formats, use better algorithms, or use C/Numba
accelerated code than it is to parallelize. Unfortunately storage formats
and C aren’t as sexy as big data parallelism, so they’re not in the forefront
of people’s minds. We should change this.&lt;/p&gt;
&lt;p&gt;I’ll proudly buy a beer for anyone that helps to make storage formats a sexier
topic.&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/2015/12/21/dask-year.md&lt;/span&gt;, line 142)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="scheduling"&gt;
&lt;h1&gt;Scheduling&lt;/h1&gt;
&lt;section id="single-machine"&gt;
&lt;h2&gt;Single Machine&lt;/h2&gt;
&lt;p&gt;The single machine dynamic task scheduler is very very solid. It has roughly
two objectives:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Use all the cores of a machine&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Choose tasks that allow the release of intermediate results&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This is what allows us to quickly execute complex workflows in small space.
This scheduler underlies all execution within dask. I’m very happy with it. I
would like to find ways to expose it more broadly to other libraries.
Suggestions are very welcome here.&lt;/p&gt;
&lt;p&gt;We still run into cases where it doesn’t perform optimally
(see &lt;a class="reference external" href="https://github.com/blaze/dask/issues/874"&gt;issue 874&lt;/a&gt;),
but so far we’ve always been able to enhance the scheduler whenever these cases
arise.&lt;/p&gt;
&lt;/section&gt;
&lt;section id="distributed-cluster"&gt;
&lt;h2&gt;Distributed Cluster&lt;/h2&gt;
&lt;p&gt;Over the last few months we’ve been working on another scheduler for
distributed memory computation. It should be a nice extension to the existing
dask collections out to “big data” systems. It’s experimental but usable now
with documentation at the follow links:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.org/en/latest/"&gt;http://distributed.readthedocs.org/en/latest/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="github reference external" href="https://github.com/blaze/distributed"&gt;blaze/distributed&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://matthewrocklin.com/distributed-by-example/"&gt;http://matthewrocklin.com/distributed-by-example/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Feedback is welcome. I recommend waiting for a month or two if you prefer
clean and reliable software. It will undergo a name-change to something less
generic.&lt;/p&gt;
&lt;/section&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/12/21/dask-year/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="dask" label="dask"/>
    <published>2015-12-21T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/10/09/Distributed/</id>
    <title>Distributed Prototype</title>
    <updated>2015-10-09T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr: We demonstrate a prototype distributed computing library and discuss
data locality.&lt;/strong&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/2015/10/09/Distributed.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="distributed-computing"&gt;

&lt;p&gt;Here’s a new prototype library for distributed computing.
It could use some critical feedback.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.org/en/latest/"&gt;distributed.readthedocs.org/en/latest/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://github.com/mrocklin/distributed/"&gt;github.com/mrocklin/distributed/&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;This blogpost uses &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed&lt;/span&gt;&lt;/code&gt; on a toy example. I won’t talk about the
design here, but the docs should be a quick and informative read. I recommend
the &lt;a class="reference external" href="http://distributed.readthedocs.org/en/latest/quickstart.html"&gt;quickstart&lt;/a&gt;
in particular.&lt;/p&gt;
&lt;p&gt;We’re going to do a simple computation a few different ways on a cluster of
four nodes. The computation will be&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Make a 1000 random numpy arrays, each of size 1 000 000&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Compute the sum of each array&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Compute the total sum of the sums&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We’ll do this directly with a distributed Pool and again with a dask graph.&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/2015/10/09/Distributed.md&lt;/span&gt;, line 37)&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="start-up-a-cluster"&gt;
&lt;h1&gt;Start up a Cluster&lt;/h1&gt;
&lt;p&gt;I have a cluster of four &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;m3.xlarge&lt;/span&gt;&lt;/code&gt;s on EC2&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;ssh node1
dcenter

ssh node2
dworkder node1:8787

ssh node3
dworkder node1:8787

ssh node4
dworkder node1:8787
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;&lt;a class="reference external" href="https://gist.github.com/mrocklin/3c1e47f403490edb9473"&gt;Notes on how I set up my cluster.&lt;/a&gt;&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/10/09/Distributed.md&lt;/span&gt;, line 55)&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="pool"&gt;
&lt;h1&gt;Pool&lt;/h1&gt;
&lt;p&gt;On the client side we spin up a distributed &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pool&lt;/span&gt;&lt;/code&gt; and point it to the center
node.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="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;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;Pool&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;pool&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Pool&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;node1:8787&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Then we create a bunch of random numpy arrays:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;,&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="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Our result is a list of proxy objects that point back to individual numpy arrays
on the worker computers. We don’t move data until we need to. (Though we
could call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.get()&lt;/span&gt;&lt;/code&gt; on this to collect the numpy array from the worker.)&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;arrays&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="go"&gt;RemoteData&amp;lt;center=10.141.199.202:8787, key=3e446310-6...&amp;gt;&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Further computations on this data happen on the cluster, on the worker nodes
that hold the data already.&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;sums&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&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;arrays&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This avoids costly data transfer times. Data transfer will happen when
necessary though, as when we compute the final sum. This forces communication
because all of the intermediate sums must move to one node for the final
addition.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;total&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;apply&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&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;args&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sums&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;total&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="c1"&gt;# finally transfer result to local machine&lt;/span&gt;
&lt;span class="go"&gt;499853416.82058007&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/2015/10/09/Distributed.md&lt;/span&gt;, line 99)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="distributed-dask"&gt;
&lt;h1&gt;distributed.dask&lt;/h1&gt;
&lt;p&gt;Now we do the same computation all at once by manually constructing a dask
graph (beware, this can get gnarly, friendlier approaches exist 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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;dsk&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;dict&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="k"&gt;for&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;range&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="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;dsk&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;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="gp"&gt;... &lt;/span&gt;    &lt;span class="n"&gt;dsk&lt;/span&gt;&lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;sum&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;i&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&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="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;i&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;dsk&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;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;sum&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;i&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="ow"&gt;in&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;1000&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="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;distributed.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;get&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;node1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;8787&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;dsk&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="go"&gt;500004095.00759566&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Apparently not everyone finds dask dictionaries to be pleasant to write by
hand. You could also use this with dask.imperative or dask.array.&lt;/p&gt;
&lt;section id="dask-imperative"&gt;
&lt;h2&gt;dask.imperative&lt;/h2&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;get2&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dsk&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;keys&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; Make `get` scheduler that hardcodes the IP and Port &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;node1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;8787&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;dsk&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;keys&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&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.imperative&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;do&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;do&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;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="k"&gt;for&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;range&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;sums&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;do&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&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="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;arrays&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;total&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;do&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="n"&gt;sums&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;total&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;get&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;get2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;499993637.00844824&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/section&gt;
&lt;section id="dask-array"&gt;
&lt;h2&gt;dask.array&lt;/h2&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.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1000000&lt;/span&gt;&lt;span class="o"&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;1000000&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;get2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;500000250.44921482&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The dask approach was smart enough to delete all of the intermediates that it
didn’t need. It could have run intelligently on far more data than even our
cluster could hold. With the pool we manage data ourselves manually.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;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;delete&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;delete&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;node0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;8787&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;arrays&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/2015/10/09/Distributed.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&gt;
&lt;section id="mix-and-match"&gt;
&lt;h1&gt;Mix and Match&lt;/h1&gt;
&lt;p&gt;We can also mix these abstractions and put the results from the pool into dask
graphs.&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;arrays&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pool&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;random&lt;/span&gt;&lt;span class="p"&gt;,&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="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;1000&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;dsk&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;sum&amp;#39;&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="n"&gt;np&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;x&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;x&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;arrays&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;dsk&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;=&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;sum&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;sum&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;i&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="ow"&gt;in&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;1000&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/2015/10/09/Distributed.md&lt;/span&gt;, line 167)&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="discussion"&gt;
&lt;h1&gt;Discussion&lt;/h1&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Pool&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;get&lt;/span&gt;&lt;/code&gt; user interfaces are independent from each other but both
use the same underlying network and both build off of the same codebase. With
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed&lt;/span&gt;&lt;/code&gt; I wanted to build a system that would allow me to experiment
easily. I’m mostly happy with the result so far.&lt;/p&gt;
&lt;p&gt;One non-trivial theme here is data-locality. We keep intermediate results on
the cluster and schedule jobs on computers that already have the relevant data
if possible. The workers can communicate with each other if necessary so that
any worker can do any job, but we try to arrange jobs so that workers don’t
have to communicate if not necessary.&lt;/p&gt;
&lt;p&gt;Another non-trivial aspect is that the high level &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; example works
without any tweaking of dask. Dask’s separation of schedulers from collections
means that existing dask.array code (or dask.dataframe, dask.bag,
dask.imperative code) gets to evolve as we experiment with new fancier
schedulers.&lt;/p&gt;
&lt;p&gt;Finally, I hope that the cluster setup here feels pretty minimal. You do need
some way to run a command on a bunch of machines but most people with clusters
have some mechanism to do that, even if its just ssh as I did above. My hope
is that &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;distributed&lt;/span&gt;&lt;/code&gt; lowers the bar for non-trivial cluster computing in
Python.&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/2015/10/09/Distributed.md&lt;/span&gt;, line 192)&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="disclaimer"&gt;
&lt;h1&gt;Disclaimer&lt;/h1&gt;
&lt;p&gt;Everything here is &lt;em&gt;very experimental&lt;/em&gt;. The library itself is broken
and unstable. It was made in the last few weeks and hasn’t been used on
anything serious. Please adjust expectations accordingly and
&lt;a class="reference external" href="https://github.com/mrocklin/distributed/pull/3"&gt;provide critical feedback.&lt;/a&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://distributed.readthedocs.org/en/latest/"&gt;Distributed Documentation&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/10/09/Distributed/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-10-09T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/08/03/Caching/</id>
    <title>Caching</title>
    <updated>2015-08-03T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr: Caching improves performance under repetitive workloads. Traditional
LRU policies don’t fit data science well. We propose a new caching policy.&lt;/strong&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/2015/08/03/Caching.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 id="humans-repeat-stuff"&gt;

&lt;p&gt;Consider the dataset that you’ve worked on most recently. How many times have
you loaded it from disk into memory? How many times have you repeated almost
the same computations on that data?&lt;/p&gt;
&lt;p&gt;Exploratory data science workloads involve repetition of very similar
computations. These computations share structure. By caching frequently used
results (or parts of results) we may be able to considerably speed up
exploratory data analysis.&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/2015/08/03/Caching.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="caching-in-other-contexts"&gt;
&lt;h1&gt;Caching in other contexts&lt;/h1&gt;
&lt;p&gt;The web community loves caching. Database backed web applications almost
always guard their database lookups with a system like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;memcached&lt;/span&gt;&lt;/code&gt; which
devotes some amount of memory to caching frequent and recent queries.
Because humans visit mostly the same pages over and over again this can reduce
database load by an order of magnitude. Even if humans visit different pages
with different inputs these pages often share many elements.&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/2015/08/03/Caching.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="limited-caching-resources"&gt;
&lt;h1&gt;Limited caching resources&lt;/h1&gt;
&lt;p&gt;Given infinite memory we would cache every result that we’ve ever
computed. This would give us for instant recall on anything that wasn’t novel.
Sadly our memory resources are finite and so we evict cached results
that don’t seem to be worth keeping around.&lt;/p&gt;
&lt;p&gt;Traditionally we use a policy like Least Recently Used (LRU). This policy
evicts results that have not been requested for a long time. This is cheap and
works well for web and systems applications.&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/2015/08/03/Caching.md&lt;/span&gt;, line 47)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="lru-doesn-t-fit-analytic-workloads"&gt;
&lt;h1&gt;LRU doesn’t fit analytic workloads&lt;/h1&gt;
&lt;p&gt;Unfortunately LRU doesn’t fit analytic workloads well. Analytic workloads have
a large spread computation times and of storage costs. While most web
application database queries take roughly the same amount of time
(100ms-1000ms) and take up roughly the same amount of space to store (1-10kb),
the computation and storage costs of analytic computations can easily vary by
many orders of magnitude (spreads in the millions or billions are common.)&lt;/p&gt;
&lt;p&gt;Consider the following two common computations of a large NumPy array:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x.std()&lt;/span&gt; &lt;span class="pre"&gt;#&lt;/span&gt; &lt;span class="pre"&gt;costly&lt;/span&gt; &lt;span class="pre"&gt;to&lt;/span&gt; &lt;span class="pre"&gt;recompute,&lt;/span&gt; &lt;span class="pre"&gt;cheap&lt;/span&gt; &lt;span class="pre"&gt;to&lt;/span&gt; &lt;span class="pre"&gt;store&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x.T&lt;/span&gt; &lt;span class="pre"&gt;#&lt;/span&gt; &lt;span class="pre"&gt;cheap&lt;/span&gt; &lt;span class="pre"&gt;to&lt;/span&gt; &lt;span class="pre"&gt;recompute,&lt;/span&gt; &lt;span class="pre"&gt;costly&lt;/span&gt; &lt;span class="pre"&gt;to&lt;/span&gt; &lt;span class="pre"&gt;store&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;In the first case, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x.std()&lt;/span&gt;&lt;/code&gt;, this might take a second on a large array
(somewhat expensive) but takes only a few bytes to store. This result is so
cheap to store that we’re happy to keep it in our cache for a long time, even
if its infrequently requested.&lt;/p&gt;
&lt;p&gt;In the second case, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x.T&lt;/span&gt;&lt;/code&gt; this is cheap to compute (just a metadata change in
the array) and executes in microseconds. However the result might take
gigabytes of memory to store. We don’t want to keep this in our cache, even if
it’s very frequently requested; it takes up all of the space for other
potentially more useful (and smaller) results and we can recompute it trivially
anyway.&lt;/p&gt;
&lt;p&gt;So we want to keep cached results that have the following properties:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Costly to recompute (in seconds)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Cheap to store (in bytes)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Frequently used&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Recently used&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/08/03/Caching.md&lt;/span&gt;, line 80)&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="proposed-caching-policy"&gt;
&lt;h1&gt;Proposed Caching Policy&lt;/h1&gt;
&lt;p&gt;Here is an alternative to LRU that respects the objectives stated above.&lt;/p&gt;
&lt;p&gt;Every time someone accesses an entry in our cache, we increment the score
associated to the entry with the following value&lt;/p&gt;
&lt;div class="math notranslate nohighlight"&gt;
\[ \textrm{score} = \textrm{score} + \frac{\textrm{compute time}}{\textrm{nbytes}} (1 + \epsilon)^{t} \]&lt;/div&gt;
&lt;p&gt;Where &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;compute&lt;/span&gt; &lt;span class="pre"&gt;time&lt;/span&gt;&lt;/code&gt; is the time it took to compute the result in the first
place, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;nbytes&lt;/span&gt;&lt;/code&gt; is the number of bytes that it takes to store the result,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;epsilon&lt;/span&gt;&lt;/code&gt; is a small number that determines the halflife of what “recently”
means, and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;t&lt;/span&gt;&lt;/code&gt; is an auto-incrementing tick time increased at every access.&lt;/p&gt;
&lt;p&gt;This has units of inverse bandwidth (s/byte), gives more importance to new
results with a slowly growing exponential growth, and amplifies the score of
frequently requested results in a roughly linear fashion.&lt;/p&gt;
&lt;p&gt;We maintain these scores in a heap, keep track of the total number of bytes,
and cull the cache as necessary to keep storage costs beneath a fixed budget.
Updates cost &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;O(log(k))&lt;/span&gt;&lt;/code&gt; for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;k&lt;/span&gt;&lt;/code&gt; the number of elements in the cache.&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/2015/08/03/Caching.md&lt;/span&gt;, line 102)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="cachey"&gt;
&lt;h1&gt;Cachey&lt;/h1&gt;
&lt;p&gt;I wrote this up into a tiny library called &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cachey&lt;/span&gt;&lt;/code&gt;. This is experimental
code and subject to wild API changes (including renaming.)&lt;/p&gt;
&lt;p&gt;The central object is a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Cache&lt;/span&gt;&lt;/code&gt; that includes asks for the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Number of available bytes to devote to the cache&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Halflife on importance (the number of access that occur to reduce the
importance of a cached result by half) (default 1000)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A lower limit on costs to consider entry to the cache (default 0)&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/08/03/Caching.md&lt;/span&gt;, line 114)&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="example"&gt;
&lt;h1&gt;Example&lt;/h1&gt;
&lt;p&gt;So here is the tiny example&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cachey&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;Cache&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Cache&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;available_bytes&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1e9&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# 1 GB&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;put&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Hello&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;world!&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cost&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;Hello&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;&amp;#39;world!&amp;#39;&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/2015/08/03/Caching.md&lt;/span&gt;, line 128)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="more-interesting-example"&gt;
&lt;h1&gt;More interesting example&lt;/h1&gt;
&lt;p&gt;The cache includes a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;memoize&lt;/span&gt;&lt;/code&gt; decorator. Lets memoize &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pd.read_csv&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;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;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;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;cachey&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;Cache&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Cache&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mf"&gt;1e9&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;read_csv&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;memoize&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_csv&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;5&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;accounts.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&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;262&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="mf"&gt;27.7&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;290&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;303&lt;/span&gt; &lt;span class="n"&gt;ms&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;6&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;df&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;accounts.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# second read is free&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;77&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;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;16&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;total&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;93&lt;/span&gt; &lt;span class="n"&gt;µs&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;93&lt;/span&gt; &lt;span class="n"&gt;µs&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;7&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;total_bytes&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;available_bytes&lt;/span&gt;
&lt;span class="n"&gt;Out&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;7&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="mf"&gt;0.096&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So we create a new function &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_csv&lt;/span&gt;&lt;/code&gt; that operates exactly like
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pandas.read_csv&lt;/span&gt;&lt;/code&gt; except that it holds on to recent results in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;c&lt;/span&gt;&lt;/code&gt;, a cache.
This particular CSV file created a dataframe that filled a tenth of our
cache space. The more often we request this CSV file the more its score will
grow and the more likely it is to remain in the cache into the future. If
other memoized functions using this same cache produce more valuable results
(costly to compute, cheap to store) and we run out of space then this result
will be evicted and we’ll have to recompute our result if we ask for
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_csv('accounts.csv')&lt;/span&gt;&lt;/code&gt; again.&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/2015/08/03/Caching.md&lt;/span&gt;, line 161)&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="memoize-everything"&gt;
&lt;h1&gt;Memoize everything&lt;/h1&gt;
&lt;p&gt;Just memoizing &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_csv&lt;/span&gt;&lt;/code&gt; isn’t very interesting. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pd.read_csv&lt;/span&gt;&lt;/code&gt;
function operates at a constant data bandwidth of around 100 MB/s. The caching
policies around &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cachey&lt;/span&gt;&lt;/code&gt; really shine when they get to see &lt;em&gt;all of our
computations&lt;/em&gt;. For example it could be that we don’t want to hold on to the
results of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;read_csv&lt;/span&gt;&lt;/code&gt; because these take up a lot of space. If we find
ourselves doing the same &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby&lt;/span&gt;&lt;/code&gt; computations then we might prefer to use our
gigabyte of caching space to store these both because&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;groupby computations take a long time to compute&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;groupby results are often very compact in memory&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/08/03/Caching.md&lt;/span&gt;, line 174)&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="cachey-and-dask"&gt;
&lt;h1&gt;Cachey and Dask&lt;/h1&gt;
&lt;p&gt;I’m slowly working on integrating cachey into dask’s shared memory scheduler.&lt;/p&gt;
&lt;p&gt;Dask is in a good position to apply &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cachey&lt;/span&gt;&lt;/code&gt; to many computations. It can look
at every task in a task graph and consider the result for inclusion into the
cache. We don’t need to explicitly &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;memoize&lt;/span&gt;&lt;/code&gt; every function we want to use,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; can do this for us on the fly.&lt;/p&gt;
&lt;p&gt;Additionally dask has a nice view of our computation as a collection of
sub-tasks. Similar computations (like mean and variance) often share
sub-tasks.&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/2015/08/03/Caching.md&lt;/span&gt;, line 187)&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;Cachey is new and untested but potentially useful now, particularly through the
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;memoize&lt;/span&gt;&lt;/code&gt; method shown above.
It’s a &lt;a class="reference external" href="http://github.com/mrocklin/cachey"&gt;small and simple codebase&lt;/a&gt;.
I would love to hear if people find this kind of caching policy valuable.&lt;/p&gt;
&lt;p&gt;I plan to think about the following in the near future:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;How do we build a hierarchy of caches to share between memory and disk?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How do we cleanly integrate cachey into dask
(see &lt;a class="reference external" href="https://github.com/ContinuumIO/dask/pull/502"&gt;PR #502&lt;/a&gt;)
and how do we make dask amenable to caching
(see &lt;a class="reference external" href="https://github.com/ContinuumIO/dask/pull/510"&gt;PR #510&lt;/a&gt;)?&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/08/03/Caching/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-08-03T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/07/23/Imperative/</id>
    <title>Custom Parallel Workflows</title>
    <updated>2015-07-23T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr: We motivate the expansion of parallel programming beyond big
collections. We discuss the usability custom of dask graphs.&lt;/strong&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/2015/07/23/Imperative.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 id="recent-parallel-work-focuses-on-big-collections"&gt;

&lt;p&gt;Parallel databases, Spark, and Dask collections all provide large distributed
collections that handle parallel computation for you. You put data into the
collection, program with a small set of operations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;groupby&lt;/span&gt;&lt;/code&gt;, and
the collections handle the parallel processing. This idea has become so
popular that there are now a dozen projects promising big and friendly Pandas
clones.&lt;/p&gt;
&lt;p&gt;This is good. These collections provide usable, high-level interfaces for a
large class of common problems.&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/2015/07/23/Imperative.md&lt;/span&gt;, line 28)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="custom-workloads"&gt;
&lt;h1&gt;Custom Workloads&lt;/h1&gt;
&lt;p&gt;However, many workloads are too complex for these collections. Workloads might
be complex either because they come from sophisticated algorithms
(as we saw in a &lt;a class="reference internal" href="#/work/2015/06/26/Complex-Graphs/"&gt;&lt;span class="xref myst"&gt;recent post on SVD&lt;/span&gt;&lt;/a&gt;) or because they come from the real world,
where problems tend to be messy.&lt;/p&gt;
&lt;p&gt;In these cases I tend to see people do two things&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Fall back to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;multiprocessing&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;MPI&lt;/span&gt;&lt;/code&gt; or some other explicit form of parallelism&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Perform mental gymnastics to fit their problem into Spark using a
clever choice of keys. These cases often fail to acheive much speedup.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/07/23/Imperative.md&lt;/span&gt;, line 41)&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="direct-dask-graphs"&gt;
&lt;h1&gt;Direct Dask Graphs&lt;/h1&gt;
&lt;p&gt;Historically I’ve recommended the manual construction of dask graphs in these
cases. Manual construction of dask graphs lets you specify fairly arbitrary
workloads that then use the dask schedulers to execute in parallel.
The &lt;a class="reference internal" href="#dask.pydata.org/en/latest/custom-graphs.html"&gt;&lt;span class="xref myst"&gt;dask docs&lt;/span&gt;&lt;/a&gt; hold the
following example of a simple data processing pipeline:&lt;/p&gt;
&lt;img src="/images/pipeline.png" align="right" width="15%"&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;load&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;filename&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="o"&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;clean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;data&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="o"&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;analyze&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;sequence_of_data&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="o"&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;store&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
    &lt;span class="o"&gt;...&lt;/span&gt;

&lt;span class="n"&gt;dsk&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;load-1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;load&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;myfile.a.data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
       &lt;span class="s1"&gt;&amp;#39;load-2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;load&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;myfile.b.data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
       &lt;span class="s1"&gt;&amp;#39;load-3&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;load&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;myfile.c.data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
       &lt;span class="s1"&gt;&amp;#39;clean-1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;clean&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;load-1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
       &lt;span class="s1"&gt;&amp;#39;clean-2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;clean&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;load-2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
       &lt;span class="s1"&gt;&amp;#39;clean-3&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;clean&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;load-3&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
       &lt;span class="s1"&gt;&amp;#39;analyze&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;analyze&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;clean-&lt;/span&gt;&lt;span class="si"&gt;%d&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;&lt;/span&gt; &lt;span class="o"&gt;%&lt;/span&gt; &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="k"&gt;for&lt;/span&gt; &lt;span class="n"&gt;i&lt;/span&gt; &lt;span class="ow"&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="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="s1"&gt;&amp;#39;store&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;store&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;analyze&amp;#39;&lt;/span&gt;&lt;span class="p"&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.multiprocessing&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;get&lt;/span&gt;
&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dsk&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;store&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# executes in parallel&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Feedback from users is that this is interesting and powerful but that
programming directly in dictionaries is not inutitive, doesn’t integrate well
with IDEs, and is prone to error.&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/2015/07/23/Imperative.md&lt;/span&gt;, line 78)&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="introducing-dask-do"&gt;
&lt;h1&gt;Introducing dask.do&lt;/h1&gt;
&lt;p&gt;To create the same custom parallel workloads using normal-ish Python code we
use the &lt;a class="reference external" href="http://dask.pydata.org/en/latest/imperative.html"&gt;dask.do&lt;/a&gt; function.
This &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;do&lt;/span&gt;&lt;/code&gt; function turns any normal Python function into a delayed version that
adds to a dask graph. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;do&lt;/span&gt;&lt;/code&gt; function lets us rewrite the computation above
as follows:&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&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;do&lt;/span&gt;

&lt;span class="n"&gt;loads&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;do&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;load&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.a.data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
         &lt;span class="n"&gt;do&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;load&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.b.data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
         &lt;span class="n"&gt;do&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;load&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.c.data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;

&lt;span class="n"&gt;cleaned&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;do&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;clean&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="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;loads&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;

&lt;span class="n"&gt;analysis&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;do&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;analyze&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="n"&gt;cleaned&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="n"&gt;do&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;store&lt;/span&gt;&lt;span class="p"&gt;)(&lt;/span&gt;&lt;span class="n"&gt;analysis&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The explicit function calls here don’t perform work directly; instead they
build up a dask graph which we can then execute in parallel with our choice of
scheduler.&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.multiprocessing&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;get&lt;/span&gt;
&lt;span class="n"&gt;result&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;get&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This interface was suggested by &lt;a class="reference external" href="http://gael-varoquaux.info/"&gt;Gael Varoquaux&lt;/a&gt;
based on his experience with &lt;a class="reference external" href="https://pythonhosted.org/joblib/"&gt;joblib&lt;/a&gt;. It
was implemented by &lt;a class="reference external" href="http://jcrist.github.io/"&gt;Jim Crist&lt;/a&gt;
in &lt;a class="reference external" href="https://github.com/ContinuumIO/dask/pull/408"&gt;PR (#408)&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/07/23/Imperative.md&lt;/span&gt;, line 113)&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="example-nested-cross-validation"&gt;
&lt;h1&gt;Example: Nested Cross Validation&lt;/h1&gt;
&lt;p&gt;I sat down with a Machine learning student, &lt;a class="reference external" href="http://people.inf.ethz.ch/kgabriel/"&gt;Gabriel
Krummenacher&lt;/a&gt; and worked to parallelize a
small code to do nested cross validation. Below is a comparison of a
sequential implementation that has been parallelized using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.do&lt;/span&gt;&lt;/code&gt;:&lt;/p&gt;
&lt;p&gt;You can safely skip reading this code in depth. The take-away is that it’s
somewhat involved but that the addition of parallelism is light.&lt;/p&gt;
&lt;a href="https://github.com/mrocklin/dask-crossval"&gt;
&lt;img src="/images/do.gif" alt="parallized cross validation code"
     width="80%"&gt;
&lt;/a&gt;
&lt;p&gt;The parallel version runs about four times faster on my notebook.
Disclaimer: The sequential version presented here is just a downgraded version
of the parallel code, hence why they look so similar. This is available
&lt;a class="reference external" href="http://github.com/mrocklin/dask-crossval"&gt;on github&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;So the result of our normal imperative-style for-loop code is a fully
parallelizable dask graph. We visualize that graph below.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;test_score.visualize()
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/crossval.png"&gt;
  &lt;img src="/images/crossval.png"
       alt="Cross validation dask graph"&gt;
&lt;/a&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/2015/07/23/Imperative.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="help"&gt;
&lt;h1&gt;Help&lt;/h1&gt;
&lt;p&gt;Is this a useful interface? It would be great if people could try this out
and &lt;a class="reference external" href="http://github.com/ContinuumIO/dask/issues/new"&gt;generate feedback&lt;/a&gt; on &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.do&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;For more information on &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.do&lt;/span&gt;&lt;/code&gt; see the
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/imperative.html"&gt;dask imperative documentation&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/07/23/Imperative/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-07-23T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/06/26/Complex-Graphs/</id>
    <title>Write Complex Parallel Algorithms</title>
    <updated>2015-06-26T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr: We discuss the use of complex dask graphs for non-trivial algorithms.
We show off an on-disk parallel SVD.&lt;/strong&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/2015/06/26/Complex-Graphs.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 id="most-parallel-computation-is-simple"&gt;

&lt;p&gt;Most parallel workloads today are fairly trivial:&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.bag&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;db&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;db&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_s3&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;githubarchive-data&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2015-01-01-*.json.gz&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;          .map(json.loads)&lt;/span&gt;
&lt;span class="go"&gt;          .map(lambda d: d[&amp;#39;type&amp;#39;] == &amp;#39;PushEvent&amp;#39;)&lt;/span&gt;
&lt;span class="go"&gt;          .count()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Graphs for these computations look like the following:&lt;/p&gt;
&lt;a href="/images/dask-bag-embarassing.png"&gt;
&lt;img src="/images/dask-bag-embarassing.png"
     width="70%"
     alt="Embarrassingly parallel dask graph"&gt;&lt;/a&gt;
&lt;p&gt;This is great; these are simple problems to solve efficiently in parallel.
Generally these simple computations occur at the &lt;em&gt;beginning&lt;/em&gt; of our analyses.&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/2015/06/26/Complex-Graphs.md&lt;/span&gt;, line 38)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="sophisticated-algorithms-can-be-complex"&gt;
&lt;h1&gt;Sophisticated Algorithms can be Complex&lt;/h1&gt;
&lt;p&gt;Later in our analyses we want more complex algorithms for statistics
, machine learning, etc.. Often this stage fits
comfortably in memory, so we don’t worry about parallelism and can use
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;statsmodels&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;scikit-learn&lt;/span&gt;&lt;/code&gt; on the gigabyte result we’ve gleaned from
terabytes of data.&lt;/p&gt;
&lt;p&gt;However, if our reduced result is still large then we need to think about
sophisticated parallel algorithms. This is fresh space with lots of exciting
academic and software work.&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/2015/06/26/Complex-Graphs.md&lt;/span&gt;, line 50)&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="example-parallel-stable-out-of-core-svd"&gt;
&lt;h1&gt;Example: Parallel, Stable, Out-of-Core SVD&lt;/h1&gt;
&lt;p&gt;I’d like to show off work by &lt;a class="reference external" href="http://www.marianotepper.com.ar/"&gt;Mariano Tepper&lt;/a&gt;,
who is responsible for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array.linalg&lt;/span&gt;&lt;/code&gt;. In particular he has a couple of
wonderful algorithms for the
&lt;a class="reference external" href="https://en.wikipedia.org/wiki/Singular_value_decomposition"&gt;Singular Value Decomposition (SVD)&lt;/a&gt;
(also strongly related to &lt;a class="reference external" href="https://en.wikipedia.org/wiki/Principal_component_analysis"&gt;Principal Components Analysis (PCA)&lt;/a&gt;.)
Really I just want to show off this pretty graph.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;x&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;ones&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;5000&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;1000&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;da&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/dask-svd.png"&gt;
&lt;img src="/images/dask-svd.png"
     width="60%"
     alt="Parallel SVD dask graph"&gt;&lt;/a&gt;
&lt;p&gt;This algorithm computes the exact SVD (up to numerical precision) of a large
tall-and-skinny matrix in parallel in many small chunks. This allows it to
operate out-of-core (from disk) and use multiple cores in parallel. At the
bottom we see the construction of our trivial array of ones, followed by many
calls to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.linalg.qr&lt;/span&gt;&lt;/code&gt; on each of the blocks. Then there is a lot of
rearranging of various pieces as they are stacked, multiplied, and undergo more
rounds of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.linalg.qr&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.linalg.svd&lt;/span&gt;&lt;/code&gt;. The resulting arrays are
available in many chunks at the top and second-from-top rows.&lt;/p&gt;
&lt;p&gt;The &lt;a class="reference external" href="http://dask.pydata.org/en/latest/spec.html"&gt;dask dict&lt;/a&gt; for one of these
arrays, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;s&lt;/span&gt;&lt;/code&gt;, looks like the following:&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;
&lt;span class="go"&gt;{(&amp;#39;x&amp;#39;, 0, 0): (np.ones, (1000, 1000)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 1, 0): (np.ones, (1000, 1000)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 2, 0): (np.ones, (1000, 1000)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 3, 0): (np.ones, (1000, 1000)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 4, 0): (np.ones, (1000, 1000)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_QR_st1&amp;#39;, 0, 0): (np.linalg.qr, (&amp;#39;x&amp;#39;, 0, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_QR_st1&amp;#39;, 1, 0): (np.linalg.qr, (&amp;#39;x&amp;#39;, 1, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_QR_st1&amp;#39;, 2, 0): (np.linalg.qr, (&amp;#39;x&amp;#39;, 2, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_QR_st1&amp;#39;, 3, 0): (np.linalg.qr, (&amp;#39;x&amp;#39;, 3, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_QR_st1&amp;#39;, 4, 0): (np.linalg.qr, (&amp;#39;x&amp;#39;, 4, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_R&amp;#39;, 0, 0): (operator.getitem, (&amp;#39;tsqr_2_QR_st2&amp;#39;, 0, 0), 1),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_R_st1&amp;#39;, 0, 0): (operator.getitem,(&amp;#39;tsqr_2_QR_st1&amp;#39;, 0, 0), 1),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_R_st1&amp;#39;, 1, 0): (operator.getitem, (&amp;#39;tsqr_2_QR_st1&amp;#39;, 1, 0), 1),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_R_st1&amp;#39;, 2, 0): (operator.getitem, (&amp;#39;tsqr_2_QR_st1&amp;#39;, 2, 0), 1),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_R_st1&amp;#39;, 3, 0): (operator.getitem, (&amp;#39;tsqr_2_QR_st1&amp;#39;, 3, 0), 1),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_R_st1&amp;#39;, 4, 0): (operator.getitem, (&amp;#39;tsqr_2_QR_st1&amp;#39;, 4, 0), 1),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_R_st1_stacked&amp;#39;, 0, 0): (np.vstack,&lt;/span&gt;
&lt;span class="go"&gt;                                   [(&amp;#39;tsqr_2_R_st1&amp;#39;, 0, 0),&lt;/span&gt;
&lt;span class="go"&gt;                                    (&amp;#39;tsqr_2_R_st1&amp;#39;, 1, 0),&lt;/span&gt;
&lt;span class="go"&gt;                                    (&amp;#39;tsqr_2_R_st1&amp;#39;, 2, 0),&lt;/span&gt;
&lt;span class="go"&gt;                                    (&amp;#39;tsqr_2_R_st1&amp;#39;, 3, 0),&lt;/span&gt;
&lt;span class="go"&gt;                                    (&amp;#39;tsqr_2_R_st1&amp;#39;, 4, 0)])),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_QR_st2&amp;#39;, 0, 0): (np.linalg.qr, (&amp;#39;tsqr_2_R_st1_stacked&amp;#39;, 0, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_SVD_st2&amp;#39;, 0, 0): (np.linalg.svd, (&amp;#39;tsqr_2_R&amp;#39;, 0, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;tsqr_2_S&amp;#39;, 0): (operator.getitem, (&amp;#39;tsqr_2_SVD_st2&amp;#39;, 0, 0), 1)}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So to write complex parallel algorithms we write down dictionaries of tuples of
functions.&lt;/p&gt;
&lt;p&gt;The dask schedulers take care of executing this graph in parallel using
multiple threads. Here is a profile result of a larger computation on a
30000x1000 array:&lt;/p&gt;
&lt;iframe src="/images/svd.profile.html"
        marginwidth="0" marginheight="0" scrolling="no"
        width="800" height="300"&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/2015/06/26/Complex-Graphs.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="low-barrier-to-entry"&gt;
&lt;h1&gt;Low Barrier to Entry&lt;/h1&gt;
&lt;p&gt;Looking at this graph you may think “Wow, Mariano is awesome” and indeed he is.
However, he is more an expert at linear algebra than at Python programming.
Dask graphs (just dictionaries) are simple enough that a domain expert was able
to look at them say “Yeah, I can do that” and write down the very complex
algorithms associated to his domain, leaving the execution of those algorithms
up to the dask schedulers.&lt;/p&gt;
&lt;p&gt;You can see the source code that generates the above graphs
&lt;a class="reference external" href="https://github.com/ContinuumIO/dask/blob/master/dask/array/linalg.py"&gt;on GitHub&lt;/a&gt;.&lt;/p&gt;
&lt;a href="/images/dask-svd-random.png"&gt;
&lt;img src="/images/dask-svd-random.png"
     align="right"
     alt="Approximate SVD dask graph"
     width="40%"&gt;&lt;/a&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/2015/06/26/Complex-Graphs.md&lt;/span&gt;, line 140)&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="randomized-parallel-out-of-core-svd"&gt;
&lt;h1&gt;Randomized Parallel Out-of-Core SVD&lt;/h1&gt;
&lt;p&gt;A few weeks ago
&lt;a class="reference external" href="https://github.com/ContinuumIO/dask/issues/265"&gt;a genomics researcher asked&lt;/a&gt;
for an approximate/randomized variant to SVD.
Mariano had &lt;a class="reference external" href="https://github.com/ContinuumIO/dask/pull/280"&gt;a solution&lt;/a&gt;
up in a few days.&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.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;x&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;ones&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;5000&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;1000&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;da&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_compressed&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;k&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;100&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;n_power_iter&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;I’ll omit the full dict for obvious space reasons.&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/2015/06/26/Complex-Graphs.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="final-thoughts"&gt;
&lt;h1&gt;Final Thoughts&lt;/h1&gt;
&lt;p&gt;Dask graphs let us express parallel algorithms with very little extra
complexity. There are no special objects or frameworks to learn, just
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/spec.html"&gt;dictionaries of tuples of functions&lt;/a&gt;.
This allows domain experts to write sophisticated algorithms without fancy code
getting in their way.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/06/26/Complex-Graphs/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-06-26T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/06/23/Distributed/</id>
    <title>Distributed Scheduling</title>
    <updated>2015-06-23T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr: We evaluate dask graphs with a variety of schedulers and introduce a
new distributed memory scheduler.&lt;/strong&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Dask.distributed is new and is not battle-tested. Use at your own risk and
adjust expectations accordingly.&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/2015/06/23/Distributed.md&lt;/span&gt;, line 18)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="evaluate-dask-graphs"&gt;

&lt;p&gt;Most dask users use the dask collections, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Array&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Bag&lt;/span&gt;&lt;/code&gt;, and
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DataFrame&lt;/span&gt;&lt;/code&gt;. These collections are convenient ways to produce
dask graphs. A dask graph is a dictionary of tasks. A task is a tuple with a
function and arguments.&lt;/p&gt;
&lt;p&gt;The graph comprising a dask collection (like a dask.array) is available through
its &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;.dask&lt;/span&gt;&lt;/code&gt; attribute.&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.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;x&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;arange&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;15&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;5&lt;/span&gt;&lt;span class="p"&gt;,))&lt;/span&gt;  &lt;span class="c1"&gt;# 0..14 in three chunks of size five&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;dask&lt;/span&gt;  &lt;span class="c1"&gt;# dask array holds the graph to create the full array&lt;/span&gt;
&lt;span class="go"&gt;{(&amp;#39;x&amp;#39;, 0): (np.arange, 0, 5),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 1): (np.arange, 5, 10),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 2): (np.arange, 10, 15)}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Further operations on &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; create more complex graphs&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;z&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="mi"&gt;100&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="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;z&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;
&lt;span class="go"&gt;{(&amp;#39;x&amp;#39;, 0): (np.arange, 0, 5),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 1): (np.arange, 5, 10),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 2): (np.arange, 10, 15),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;y&amp;#39;, 0): (add, (&amp;#39;x&amp;#39;, 0), 100),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;y&amp;#39;, 1): (add, (&amp;#39;x&amp;#39;, 1), 100),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;y&amp;#39;, 2): (add, (&amp;#39;x&amp;#39;, 2), 100),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;z&amp;#39;, 0): (np.sum, (&amp;#39;y&amp;#39;, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;z&amp;#39;, 1): (np.sum, (&amp;#39;y&amp;#39;, 1)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;z&amp;#39;, 2): (np.sum, (&amp;#39;y&amp;#39;, 2)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;z&amp;#39;,): (sum, [(&amp;#39;z&amp;#39;, 0), (&amp;#39;z&amp;#39;, 1), (&amp;#39;z&amp;#39;, 2)])}&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/2015/06/23/Distributed.md&lt;/span&gt;, line 55)&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="hand-made-dask-graphs"&gt;
&lt;h1&gt;Hand-made dask graphs&lt;/h1&gt;
&lt;p&gt;We can make dask graphs by hand without dask collections. This involves
creating a dictionary of tuples of functions.&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="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;add&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="n"&gt;b&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="k"&gt;return&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;b&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="c1"&gt;# x = 1&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="c1"&gt;# y = 2&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="c1"&gt;# z = add(x, y)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;dsk&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;       &lt;span class="s1"&gt;&amp;#39;y&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;       &lt;span class="s1"&gt;&amp;#39;z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;y&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We evaluate these graphs with one of the dask schedulers&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.threaded&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;get&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dsk&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;   &lt;span class="c1"&gt;# Evaluate graph with multiple threads&lt;/span&gt;
&lt;span class="go"&gt;3&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.multiprocessing&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;get&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dsk&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;   &lt;span class="c1"&gt;# Evaluate graph with multiple processes&lt;/span&gt;
&lt;span class="go"&gt;3&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We separate the evaluation of the graphs from their construction.&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/2015/06/23/Distributed.md&lt;/span&gt;, line 87)&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="distributed-scheduling"&gt;
&lt;h1&gt;Distributed Scheduling&lt;/h1&gt;
&lt;p&gt;The separation of graphs from evaluation allows us to create new schedulers.
In particular there exists a scheduler that operates on multiple machines in
parallel, communicating over ZeroMQ.&lt;/p&gt;
&lt;p&gt;This system has a single centralized scheduler, several workers, and
potentially several clients.&lt;/p&gt;
&lt;p&gt;&lt;img alt="Workers and clients connecting to a sheduler" src="https://blog.dask.org/_images/distributed-layout.png" /&gt;&lt;/p&gt;
&lt;p&gt;Clients send graphs to the central scheduler which farms out those tasks to
workers and coordinates the execution of the graph. While the scheduler
centralizes metadata, the workers themselves handle transfer of intermediate
data in a peer-to-peer fashion. Once the graph completes the workers send data
to the scheduler which passes it through to the appropriate user/client.&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/2015/06/23/Distributed.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="example"&gt;
&lt;h1&gt;Example&lt;/h1&gt;
&lt;iframe width="560" height="315"
src="https://www.youtube.com/embed/uQro_CaP9Fo?rel=0" frameborder="0"
allowfullscreen&gt;&lt;/iframe&gt;
&lt;p&gt;And so now we can execute our dask graphs in parallel across multiple machines.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="err"&gt;$&lt;/span&gt; &lt;span class="n"&gt;ipython&lt;/span&gt;  &lt;span class="c1"&gt;# On your laptop                 $ ipython  # Remote Process #1:  Scheduler&lt;/span&gt;
&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;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;add&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="n"&gt;b&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;                          &lt;span class="o"&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;Scheduler&lt;/span&gt;
&lt;span class="o"&gt;...&lt;/span&gt;     &lt;span class="k"&gt;return&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;b&lt;/span&gt;                        &lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Scheduler&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;port_to_workers&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;4444&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
                                            &lt;span class="o"&gt;...&lt;/span&gt;               &lt;span class="n"&gt;port_to_clients&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;5555&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;dsk&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;                          &lt;span class="o"&gt;...&lt;/span&gt;               &lt;span class="n"&gt;hostname&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;notebook&amp;#39;&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;y&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="o"&gt;...&lt;/span&gt;        &lt;span class="s1"&gt;&amp;#39;z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;add&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;y&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)}&lt;/span&gt;            &lt;span class="err"&gt;$&lt;/span&gt; &lt;span class="n"&gt;ipython&lt;/span&gt;  &lt;span class="c1"&gt;# Remote Process #2:  Worker&lt;/span&gt;
                                            &lt;span class="o"&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;Worker&lt;/span&gt;
&lt;span class="o"&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.threaded&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;get&lt;/span&gt;           &lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;w&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Worker&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tcp://notebook:4444&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dsk&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# use threads&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;                                           &lt;span class="err"&gt;$&lt;/span&gt; &lt;span class="n"&gt;ipython&lt;/span&gt;  &lt;span class="c1"&gt;# Remote Process #3:  Worker&lt;/span&gt;
                                            &lt;span class="o"&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;Worker&lt;/span&gt;
                                            &lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;w&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Worker&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tcp://notebook:4444&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="o"&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="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Client&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;tcp://notebook:5555&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="o"&gt;&amp;gt;&amp;gt;&amp;gt;&lt;/span&gt; &lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dsk&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;z&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="c1"&gt;# use distributed network&lt;/span&gt;
&lt;span class="mi"&gt;3&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/06/23/Distributed.md&lt;/span&gt;, line 134)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="choose-your-scheduler"&gt;
&lt;h1&gt;Choose Your Scheduler&lt;/h1&gt;
&lt;p&gt;This graph is small. We didn’t need a distributed network of machines
to compute it (a single thread would have been much faster)
but this simple example can be easily extended to more important cases,
including generic use with the dask collections (Array, Bag, DataFrame). You
can control the scheduler with a keyword argument to any compute call.&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.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;x&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;normal&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mf"&gt;0.1&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;size&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1000000000&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;1000000&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;get&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="c1"&gt;# use threads&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;mean&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# use distributed network&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Alternatively you can set the default scheduler in dask with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.set_options&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;set_options&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;c&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;get&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# use distributed scheduler by default&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/2015/06/23/Distributed.md&lt;/span&gt;, line 157)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="known-limitations"&gt;
&lt;h1&gt;Known Limitations&lt;/h1&gt;
&lt;p&gt;We intentionally made the simplest and dumbest distributed scheduler we could
think of. Because dask separates graphs from schedulers we can iterate on this
problem many times; building better schedulers after learning what is
important. This current scheduler learns from our single-memory system but is
the first dask scheduler that has to think about distributed memory. As a
result it has the following known limitations:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;It does not consider data locality. While linear chains of tasks will
execute on the same machine we don’t think much about executing multi-input
tasks on nodes where only some of the data is local.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;In particular, this scheduler isn’t optimized for data-local file-systems
like HDFS. It’s still happy to read data from HDFS, but this results in
unnecessary network communication. We’ve found that it’s great when paired
with S3.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;This scheduler is new and hasn’t yet had its tires kicked. Vocal beta
users are most welcome.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;We haven’t thought much about deployment. E.g. somehow you need to ssh
into a bunch of machines and start up workers, then tear them down when you’re
done. Dask.distributed can bootstrap off of an
&lt;a class="reference external" href="http://dask.readthedocs.org/en/latest/distributed.html#ipython-parallel"&gt;IPython Parallel cluster&lt;/a&gt;,
and we’ve integrated it into
&lt;a class="reference external" href="http://continuum.io/anaconda-cluster"&gt;anaconda-cluster&lt;/a&gt; but deployment remains
a tough problem.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed&lt;/span&gt;&lt;/code&gt; module is available in the last release but I suggest
using the development master branch. There will be another release in early
July.&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/2015/06/23/Distributed.md&lt;/span&gt;, line 187)&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="further-information"&gt;
&lt;h1&gt;Further Information&lt;/h1&gt;
&lt;p&gt;&lt;a class="reference external" href="https://github.com/cowlicks"&gt;Blake Griffith&lt;/a&gt; has been playing with
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.bag&lt;/span&gt;&lt;/code&gt; together on data from
&lt;a class="reference external" href="http://githubarchive.org"&gt;http://githubarchive.org&lt;/a&gt;. He plans to write a
blogpost to give a better demonstration of the use of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed&lt;/span&gt;&lt;/code&gt; on
real world problems. Look for that post in the next week or two.&lt;/p&gt;
&lt;p&gt;You can read more about the internal design of &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed&lt;/span&gt;&lt;/code&gt; at the
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/distributed.html"&gt;dask docs&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/06/23/Distributed.md&lt;/span&gt;, line 198)&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="thanks"&gt;
&lt;h1&gt;Thanks&lt;/h1&gt;
&lt;p&gt;Special thanks to &lt;a class="reference external" href="https://github.com/minrk"&gt;Min Regan-Kelley&lt;/a&gt;,
&lt;a class="reference external" href="http://eigenhombre.com/"&gt;John Jacobsen&lt;/a&gt;,
&lt;a class="reference external" href="https://twitter.com/quasiben"&gt;Ben Zaitlen&lt;/a&gt;,
and &lt;a class="reference external" href="https://www.linkedin.com/pub/hugo-shi/10/579/442"&gt;Hugo Shi&lt;/a&gt;
for their advice on building distributed systems.&lt;/p&gt;
&lt;p&gt;Also thanks to &lt;a class="reference external" href="https://github.com/cowlicks"&gt;Blake Griffith&lt;/a&gt; for serving as
original user/developer and for smoothing over the user experience.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/06/23/Distributed/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-06-23T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/05/19/State-of-Dask/</id>
    <title>State of Dask</title>
    <updated>2015-05-19T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr&lt;/strong&gt; We lay out the pieces of Dask, a system for parallel computing&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/2015/05/19/State-of-Dask.md&lt;/span&gt;, line 14)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="introduction"&gt;

&lt;p&gt;Dask started five months ago as a parallel on-disk array; it has
since broadened out. I’ve enjoyed &lt;a class="reference external" href="http://matthewrocklin.com/blog/tags.html#dask-ref"&gt;writing about its
development&lt;/a&gt; tremendously.
With the &lt;a class="reference external" href="https://groups.google.com/a/continuum.io/d/topic/blaze-dev/HAoLbk2F0PI/discussion"&gt;recent 0.5.0
release&lt;/a&gt;
I decided to take a moment to give an overview of dask’s various pieces, their
state, and current development.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/05/19/State-of-Dask.md&lt;/span&gt;, line 24)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="collections-graphs-and-schedulers"&gt;
&lt;h1&gt;Collections, graphs, and schedulers&lt;/h1&gt;
&lt;p&gt;Dask modules can be separated as follows:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/collections-schedulers.png"
     width="70%"
     align="right"
     alt="Partitioned Frame design"&gt;&lt;/p&gt;
&lt;p&gt;On the left there are collections like arrays, bags, and dataframes. These
copy APIs for NumPy, PyToolz, and Pandas respectively and are aimed towards
data science users, allowing them to interact with larger datasets. Operations
on these dask collections produce task graphs which are recipes to compute the
desired result using many smaller computations that each fit in memory. For
example if we want to sum a trillion numbers then we might break the numbers
into million element chunks, sum those, and then sum the sums. A previously
impossible task becomes a million and one easy ones.&lt;/p&gt;
&lt;p&gt;On the right there are schedulers. Schedulers execute task graphs in different
situations, usually in parallel. Notably there are a few schedulers for a
single machine, and a new prototype for a &lt;a class="reference external" href="http://dask.pydata.org/en/latest/distributed.html"&gt;distributed
scheduler&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;In the center is the directed acyclic graph. This graph serves as glue between
collections and schedulers. The dask graph format is simple and doesn’t
include any dask classes; it’s just &lt;a class="reference external" href="http://dask.readthedocs.org/en/latest/spec.html"&gt;functions, dicts, and
tuples&lt;/a&gt; and so is easy to
build on and low-tech enough to understand immediately. This separation is very
useful to dask during development; improvements to one side immediately affect
the other and new developers have had surprisingly little trouble. Also
developers from a variety of backgrounds have been able to come up to speed in
about an hour.&lt;/p&gt;
&lt;p&gt;This separation is useful to other projects too. Directed acyclic graphs are
popular today in many domains. By exposing dask’s schedulers publicly, other
projects can bypass dask collections and go straight for the execution engine.&lt;/p&gt;
&lt;p&gt;A flattering quote from &lt;a class="reference external" href="https://github.com/ContinuumIO/dask/issues/153#issuecomment-92520216"&gt;a github
issue&lt;/a&gt;:&lt;/p&gt;
&lt;p&gt;&lt;em&gt;dask has been very helpful so far, as it allowed me to skip implementing
all of the usual graph operations. Especially doing the asynchronous
execution properly would have been a lot of work.&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/2015/05/19/State-of-Dask.md&lt;/span&gt;, line 68)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="who-uses-dask"&gt;
&lt;h1&gt;Who uses dask?&lt;/h1&gt;
&lt;p&gt;Dask developers work closely with a few really amazing users:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://stephanhoyer.com/"&gt;Stephan Hoyer&lt;/a&gt; at Climate Corp has integrated
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; into &lt;a class="reference internal" href="#xray.readthedocs.org"&gt;&lt;span class="xref myst"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;xray&lt;/span&gt;&lt;/code&gt;&lt;/span&gt;&lt;/a&gt; a library to manage large
volumes of meteorlogical data (and other labeled arrays.)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://scikit-image.org"&gt;Scikit image&lt;/a&gt; now includes an apply_parallel
operation (&lt;a class="reference external" href="https://github.com/scikit-image/scikit-image/pull/1493"&gt;github PR&lt;/a&gt;)
that uses dask.array to parallelize image processing routines.
(work by &lt;a class="reference external" href="https://github.com/cowlicks"&gt;Blake Griffith&lt;/a&gt;)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://www.marianotepper.com.ar/"&gt;Mariano Tepper&lt;/a&gt; a postdoc at Duke, uses
dask in his research on matrix factorizations. Mariano is also the primary
author of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array.linalg&lt;/span&gt;&lt;/code&gt; module, which includes efficient and stable
QR and SVD for tall and skinny matrices. See Mariano’s &lt;a class="reference external" href="http://arxiv.org/abs/1505.04650"&gt;paper on
arXiv&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Finally I personally use dask on daily work related to the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XData
project&lt;/a&gt;. This tends to
drive some of the newer features.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;A few other groups pop up on github from time to time; I’d love to
know more detail about how people use 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/2015/05/19/State-of-Dask.md&lt;/span&gt;, line 94)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="what-works-and-what-doesn-t"&gt;
&lt;h1&gt;What works and what doesn’t&lt;/h1&gt;
&lt;p&gt;Dask is modular. Each of the collections and each of the schedulers are
effectively separate projects. These subprojects are at different states of
development. Knowing the stability of each subproject can help you to
determine how you use and depend on dask.&lt;/p&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Dask.array&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.threaded&lt;/span&gt;&lt;/code&gt; work well, are stable, and see constant use.
They receive relatively minor bug reports which are dealt with swiftly.&lt;/p&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Dask.bag&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.multiprocessing&lt;/span&gt;&lt;/code&gt; undergo more API churn but are mostly
ready for public use with a couple of caveats. Neither &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.dataframe&lt;/span&gt;&lt;/code&gt; nor&lt;/p&gt;
&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.distributed&lt;/span&gt;&lt;/code&gt; are ready for public use; they undergo significant API churn
and have known errors.&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/2015/05/19/State-of-Dask.md&lt;/span&gt;, line 110)&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="current-work"&gt;
&lt;h1&gt;Current work&lt;/h1&gt;
&lt;p&gt;The current state of development as I see it is as follows:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Dask.bag and dask.dataframe are progressing nicely. My personal work
depends on these modules, so they see a lot of attention.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;At the moment I focus on grouping and join operations through fast
shuffles; I hope to write about this problem soon.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The Pandas API is large and complex. Reimplementing a subset of it
in a blocked way is straightforward but also detailed and time consuming.
This would be a great place for community contributions.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask.distributed is new. It needs it tires kicked but it’s an exciting
development.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;For deployment we’re planning to bootstrap off of
&lt;a class="reference external" href="http://ipython.org/ipython-doc/dev/parallel/"&gt;IPython parallel&lt;/a&gt; which
already has decent coverage of many parallel job systems,
(see &lt;a class="reference external" href="https://github.com/ContinuumIO/dask/pull/208"&gt;#208&lt;/a&gt; by Blake)&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask.array development these days focuses on outreach. We’ve found
application domains where dask is very useful; we’d like to find more.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;The collections (Array, Bag, DataFrame) don’t cover all cases. I would
like to start finding uses for the task schedulers in isolation. They
serve as a release valve in complex situations.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/05/19/State-of-Dask.md&lt;/span&gt;, line 133)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="more-information"&gt;
&lt;h1&gt;More information&lt;/h1&gt;
&lt;p&gt;You can install dask with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;conda&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;conda install dask
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;or with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pip&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;pip install dask
or
pip install dask[array]
or
pip install dask[bag]
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;You can read more about dask at &lt;a class="reference external" href="https://dask.pydata.org/en/latest/"&gt;the docs&lt;/a&gt;
or &lt;a class="reference external" href="https://github.com/ContinuumIO/dask"&gt;github&lt;/a&gt;.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/05/19/State-of-Dask/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-05-19T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/03/11/Towards-OOC-Frame/</id>
    <title>Towards Out-of-core DataFrames</title>
    <updated>2015-03-11T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;This post primarily targets developers. It is on experimental code that is
not ready for users.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr&lt;/strong&gt; Can we build &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.frame&lt;/span&gt;&lt;/code&gt;? One approach involves indexes and a lot
of shuffling.&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/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 18)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="dask-arrays-work"&gt;

&lt;p&gt;Over the last two months we’ve watched the creation of
&lt;a class="reference external" href="http://dask.pydata.org"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;, a task scheduling specification, and
&lt;a class="reference external" href="http://dask.pydata.org/en/latest/array.html"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt;&lt;/a&gt; a project to
implement the out-of-core nd-arrays using blocked algorithms.
(blogposts:
&lt;a class="reference internal" href="../../2014/12/27/Towards-OOC/"&gt;&lt;span class="doc std std-doc"&gt;1&lt;/span&gt;&lt;/a&gt;,
&lt;a class="reference internal" href="../../2014/12/30/Towards-OOC-Frontend/"&gt;&lt;span class="doc std std-doc"&gt;2&lt;/span&gt;&lt;/a&gt;,
&lt;a class="reference internal" href="../../2015/01/06/Towards-OOC-Scheduling/"&gt;&lt;span class="doc std std-doc"&gt;3&lt;/span&gt;&lt;/a&gt;,
&lt;a class="reference internal" href="../../2015/01/14/Towards-OOC-MatMul/"&gt;&lt;span class="doc std std-doc"&gt;4&lt;/span&gt;&lt;/a&gt;,
&lt;a class="reference internal" href="../../2015/01/16/Towards-OOC-SpillToDisk/"&gt;&lt;span class="doc std std-doc"&gt;5&lt;/span&gt;&lt;/a&gt;,
&lt;a class="reference internal" href="../../2015/02/13/Towards-OOC-Slicing-and-Stacking/"&gt;&lt;span class="doc std std-doc"&gt;6&lt;/span&gt;&lt;/a&gt;).
This worked pretty well. Dask.array is available on the main conda channel and on PyPI
and, for the most part, is a pleasant drop-in replacement for a subset of NumPy
operations. I’m really happy with it.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;conda install dask
or
pip install dask
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;There is still work to do, in particular I’d like to interact with people who
have real-world problems, but for the most part &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; feels ready.&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/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 42)&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="on-to-dask-frames"&gt;
&lt;h1&gt;On to dask frames&lt;/h1&gt;
&lt;p&gt;Can we do for Pandas what we’ve just done for NumPy?&lt;/p&gt;
&lt;p&gt;Question: &lt;em&gt;Can we represent a large DataFrame as a sequence of in-memory DataFrames and
perform most Pandas operations using task scheduling?&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Answer: &lt;em&gt;I don’t know. Lets try.&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/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 51)&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="naive-approach"&gt;
&lt;h1&gt;Naive Approach&lt;/h1&gt;
&lt;p&gt;If represent a dask.array as an N-d grid of NumPy ndarrays, then maybe we should
represent a dask.frame as a 1-d grid of Pandas DataFrames; they’re kind of like arrays.&lt;/p&gt;
&lt;table&gt;
&lt;th&gt;
&lt;td&gt;&lt;b&gt;dask.array&lt;/b&gt;&lt;/td&gt;
&lt;td&gt;&lt;b&gt;Naive dask.frame&lt;/b&gt;&lt;/td&gt;
&lt;/th&gt;
&lt;tr&gt;
&lt;td&gt;&lt;/td&gt;
&lt;td&gt;&lt;img src="/images/array.png"&gt;&lt;/td&gt;
&lt;td&gt;&lt;img src="/images/naive-frame.png"&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;This approach supports the following operations:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Elementwise operations &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.a&lt;/span&gt; &lt;span class="pre"&gt;+&lt;/span&gt; &lt;span class="pre"&gt;df.b&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Row-wise filtering &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df[df.a&lt;/span&gt; &lt;span class="pre"&gt;&amp;gt;&lt;/span&gt; &lt;span class="pre"&gt;0]&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Reductions &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.a.mean()&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Some split-apply-combine operations that combine with a standard reduction
like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.groupby('a').b.mean()&lt;/span&gt;&lt;/code&gt;. Essentially anything you can do with
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.groupby(...).agg(...)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;The reductions and split-apply-combine operations require some cleverness.
This is how Blaze works now and how it does the does out-of-core operations in
these notebooks:
&lt;a class="reference external" href="http://nbviewer.ipython.org/github/ContinuumIO/blaze/blob/gh-pages/notebooks/timings-csv.ipynb"&gt;Blaze and CSVs&lt;/a&gt;,
&lt;a class="reference external" href="http://nbviewer.ipython.org/github/ContinuumIO/blaze/blob/gh-pages/notebooks/timings-bcolz.ipynb"&gt;Blaze and Binary Storage&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;However this approach does not support the following operations:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Joins&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Split-apply-combine with more complex &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;transform&lt;/span&gt;&lt;/code&gt; or &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apply&lt;/span&gt;&lt;/code&gt; combine steps&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sliding window or resampling operations&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Anything involving multiple datasets&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/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 90)&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="partition-on-the-index-values"&gt;
&lt;h1&gt;Partition on the Index values&lt;/h1&gt;
&lt;p&gt;Instead of partitioning based on the size of blocks we instead partition on
value ranges of the index.&lt;/p&gt;
&lt;table&gt;
&lt;th&gt;
&lt;td&gt;&lt;b&gt;Partition on block size&lt;/b&gt;&lt;/td&gt;
&lt;td&gt;&lt;b&gt;Partition on index value&lt;/b&gt;&lt;/td&gt;
&lt;/th&gt;
&lt;tr&gt;
&lt;td&gt;&lt;/td&gt;
&lt;td&gt;&lt;img src="/images/naive-frame.png"&gt;&lt;/td&gt;
&lt;td&gt;&lt;img src="/images/frame.png"&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;This opens up a few more operations&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Joins are possible when both tables share the same index. Because we have
information about index values we we know which blocks from one side need to
communicate to which blocks from the other.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Split-apply-combine with transform/apply steps are possible when the grouper
is the index. In this case we’re guaranteed that each group is in the same
block. This opens up general &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;df.gropuby(...).apply(...)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Rolling or resampling operations are easy on the index if we share a small
amount of information between blocks as we do in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; for &lt;a class="reference external" href="http://dask.pydata.org/en/latest/ghost.html"&gt;ghosting
operations&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We note the following theme:&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Complex operations are easy if the logic aligns with the index&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;And so a recipe for many complex operations becomes:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Re-index your data along the proper column&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Perform easy computation&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 128)&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="re-indexing-out-of-core-data"&gt;
&lt;h1&gt;Re-indexing out-of-core data&lt;/h1&gt;
&lt;p&gt;To be explicit imagine we have a large time-series of transactions indexed by
time and partitioned by day. The data for every day is in a separate DataFrame.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;Block 1
-------
                     credit    name
time
2014-01-01 00:00:00     100     Bob
2014-01-01 01:00:00     200   Edith
2014-01-01 02:00:00    -300   Alice
2014-01-01 03:00:00     400     Bob
2014-01-01 04:00:00    -500  Dennis
...

Block 2
-------
                     credit    name
time
2014-01-02 00:00:00     300    Andy
2014-01-02 01:00:00     200   Edith
...
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We want to reindex this data and shuffle all of the entries so that now we
partiion on the name of the person. Perhaps all of the A’s are in one block
while all of the B’s are in another.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;Block 1
-------
                       time  credit
name
Alice   2014-04-30 00:00:00     400
Alice   2014-01-01 00:00:00     100
Andy    2014-11-12 00:00:00    -200
Andy    2014-01-18 00:00:00     400
Andy    2014-02-01 00:00:00    -800
...

Block 2
-------
                       time  credit
name
Bob     2014-02-11 00:00:00     300
Bob     2014-01-05 00:00:00     100
...
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Re-indexing and shuffling large data is difficult and expensive. We need to
find good values on which to partition our data so that we get regularly sized
blocks that fit nicely into memory. We also need to shuffle entries from all
of the original blocks to all of the new ones. In principle every old block
has something to contribute to every new one.&lt;/p&gt;
&lt;p&gt;We can’t just call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;DataFrame.sort&lt;/span&gt;&lt;/code&gt; because the entire data might not fit in
memory and most of our sorting algorithms assume random access.&lt;/p&gt;
&lt;p&gt;We do this in two steps&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Find good division values to partition our data. These should partition
the data into blocks of roughly equal size.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Shuffle our old blocks into new blocks along the new partitions found in
step one.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;img src="/images/frame-sort.png"
     align="right"
     width="30%"&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/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 195)&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="find-divisions-by-external-sorting"&gt;
&lt;h1&gt;Find divisions by external sorting&lt;/h1&gt;
&lt;p&gt;One approach to find new partition values is to pull out the new index
from each block, perform an out-of-core sort, and then take regularly spaced
values from that array.&lt;/p&gt;
&lt;ol class="arabic"&gt;
&lt;li&gt;&lt;p&gt;Pull out new index column from each block&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;indexes = [block[&amp;#39;new-column-index&amp;#39;] for block in blocks]
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Perform &lt;a class="reference internal" href="#en.wikipedia.org/wiki/External_sorting"&gt;&lt;span class="xref myst"&gt;out-of-core sort&lt;/span&gt;&lt;/a&gt; on that
column&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;sorted_index = fancy_out_of_core_sort(indexes)
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Take values at regularly spaced intervals, e.g.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;partition_values = sorted_index[::1000000]
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We implement this using parallel in-block sorts, followed by a streaming merge
process using the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;heapq&lt;/span&gt;&lt;/code&gt; module. It works but is slow.&lt;/p&gt;
&lt;section id="possible-improvements"&gt;
&lt;h2&gt;Possible Improvements&lt;/h2&gt;
&lt;p&gt;This could be accelerated through one of the following options:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A streaming numeric solution that works directly on iterators of NumPy
arrays (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numtoolz&lt;/span&gt;&lt;/code&gt; anyone?)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Not sorting at all. We only actually need approximate regularly spaced
quantiles. A brief literature search hints that there might be some good
solutions.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;&lt;img src="/images/frame-shuffle.png"
     align="right"
     width="30%"&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/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 231)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="shuffle"&gt;
&lt;h1&gt;Shuffle&lt;/h1&gt;
&lt;p&gt;Now that we know the values on which we want to partition we ask each block to
shard itself into appropriate pieces and shove all of those pieces into a
spill-to-disk dictionary. Another process then picks up these pieces and calls
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pd.concat&lt;/span&gt;&lt;/code&gt; to merge them in to the new blocks.&lt;/p&gt;
&lt;p&gt;For the out-of-core dict we’re currently using
&lt;a class="reference external" href="https://github.com/ContinuumIO/chest"&gt;Chest&lt;/a&gt;. Turns out that serializing
DataFrames and writing them to disk can be tricky. There are several good
methods with about an order of magnitude performance difference between them.&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/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 243)&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="this-works-but-my-implementation-is-slow"&gt;
&lt;h1&gt;This works but my implementation is slow&lt;/h1&gt;
&lt;p&gt;Here is an example with snippet of the NYCTaxi data (this is small)&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.frame&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;dfr&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;dfr&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;read_csv&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;/home/mrocklin/data/trip-small.csv&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;chunksize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10000&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;head&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;   &lt;span class="c1"&gt;# This is fast&lt;/span&gt;
&lt;span class="n"&gt;Out&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
                          &lt;span class="n"&gt;medallion&lt;/span&gt;                      &lt;span class="n"&gt;hack_license&lt;/span&gt;  \
&lt;span class="mi"&gt;0&lt;/span&gt;  &lt;span class="mi"&gt;89&lt;/span&gt;&lt;span class="n"&gt;D227B655E5C82AECF13C3F540D4CF4&lt;/span&gt;  &lt;span class="n"&gt;BA96DE419E711691B9445D6A6307C170&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;  &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="n"&gt;BD7C8F5BA12B88E0B67BED28BEA73D8&lt;/span&gt;  &lt;span class="mi"&gt;9&lt;/span&gt;&lt;span class="n"&gt;FD8F69F0804BDB5549F40E9DA1BE472&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;  &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="n"&gt;BD7C8F5BA12B88E0B67BED28BEA73D8&lt;/span&gt;  &lt;span class="mi"&gt;9&lt;/span&gt;&lt;span class="n"&gt;FD8F69F0804BDB5549F40E9DA1BE472&lt;/span&gt;

  &lt;span class="n"&gt;vendor_id&lt;/span&gt;  &lt;span class="n"&gt;rate_code&lt;/span&gt; &lt;span class="n"&gt;store_and_fwd_flag&lt;/span&gt;      &lt;span class="n"&gt;pickup_datetime&lt;/span&gt;  \
&lt;span class="mi"&gt;0&lt;/span&gt;       &lt;span class="n"&gt;CMT&lt;/span&gt;          &lt;span class="mi"&gt;1&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt; &lt;span class="mi"&gt;15&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;11&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;48&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;       &lt;span class="n"&gt;CMT&lt;/span&gt;          &lt;span class="mi"&gt;1&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;06&lt;/span&gt; &lt;span class="mi"&gt;00&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;18&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;35&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;       &lt;span class="n"&gt;CMT&lt;/span&gt;          &lt;span class="mi"&gt;1&lt;/span&gt;                  &lt;span class="n"&gt;N&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;05&lt;/span&gt; &lt;span class="mi"&gt;18&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;49&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;41&lt;/span&gt;

      &lt;span class="n"&gt;dropoff_datetime&lt;/span&gt;  &lt;span class="n"&gt;passenger_count&lt;/span&gt;  &lt;span class="n"&gt;trip_time_in_secs&lt;/span&gt;  &lt;span class="n"&gt;trip_distance&lt;/span&gt;  \
&lt;span class="mi"&gt;0&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt; &lt;span class="mi"&gt;15&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;18&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;                &lt;span class="mi"&gt;4&lt;/span&gt;                &lt;span class="mi"&gt;382&lt;/span&gt;            &lt;span class="mf"&gt;1.0&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;06&lt;/span&gt; &lt;span class="mi"&gt;00&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;22&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;54&lt;/span&gt;                &lt;span class="mi"&gt;1&lt;/span&gt;                &lt;span class="mi"&gt;259&lt;/span&gt;            &lt;span class="mf"&gt;1.5&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;05&lt;/span&gt; &lt;span class="mi"&gt;18&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;54&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;23&lt;/span&gt;                &lt;span class="mi"&gt;1&lt;/span&gt;                &lt;span class="mi"&gt;282&lt;/span&gt;            &lt;span class="mf"&gt;1.1&lt;/span&gt;

   &lt;span class="n"&gt;pickup_longitude&lt;/span&gt;  &lt;span class="n"&gt;pickup_latitude&lt;/span&gt;  &lt;span class="n"&gt;dropoff_longitude&lt;/span&gt;  &lt;span class="n"&gt;dropoff_latitude&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.978165&lt;/span&gt;        &lt;span class="mf"&gt;40.757977&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.989838&lt;/span&gt;         &lt;span class="mf"&gt;40.751171&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.006683&lt;/span&gt;        &lt;span class="mf"&gt;40.731781&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.994499&lt;/span&gt;         &lt;span class="mf"&gt;40.750660&lt;/span&gt;
&lt;span class="mi"&gt;2&lt;/span&gt;        &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.004707&lt;/span&gt;        &lt;span class="mf"&gt;40.737770&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.009834&lt;/span&gt;         &lt;span class="mf"&gt;40.726002&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;d2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;d&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;d&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;passenger_count&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;out_chunksize&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;10000&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;   &lt;span class="c1"&gt;# This takes some time&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;5&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;d2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;head&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;Out&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
                                        &lt;span class="n"&gt;medallion&lt;/span&gt;  \
&lt;span class="n"&gt;passenger_count&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;                &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="n"&gt;F3AC054811F8B1F095580C50FF16090&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                &lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="n"&gt;C52E48F9E05AA1A8E2F073BB932E9AA&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                &lt;span class="n"&gt;FF00E5D4B15B6E896270DDB8E0697BF7&lt;/span&gt;

                                     &lt;span class="n"&gt;hack_license&lt;/span&gt; &lt;span class="n"&gt;vendor_id&lt;/span&gt;  &lt;span class="n"&gt;rate_code&lt;/span&gt;  \
&lt;span class="n"&gt;passenger_count&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;                &lt;span class="n"&gt;E00BD74D8ADB81183F9F5295DC619515&lt;/span&gt;       &lt;span class="n"&gt;VTS&lt;/span&gt;          &lt;span class="mi"&gt;5&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                &lt;span class="mi"&gt;307&lt;/span&gt;&lt;span class="n"&gt;D1A2524E526EE08499973A4F832CF&lt;/span&gt;       &lt;span class="n"&gt;VTS&lt;/span&gt;          &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                &lt;span class="mf"&gt;0E8&lt;/span&gt;&lt;span class="n"&gt;CCD187F56B3696422278EBB620EFA&lt;/span&gt;       &lt;span class="n"&gt;VTS&lt;/span&gt;          &lt;span class="mi"&gt;1&lt;/span&gt;

                &lt;span class="n"&gt;store_and_fwd_flag&lt;/span&gt;      &lt;span class="n"&gt;pickup_datetime&lt;/span&gt;     &lt;span class="n"&gt;dropoff_datetime&lt;/span&gt;  \
&lt;span class="n"&gt;passenger_count&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;                              &lt;span class="n"&gt;NaN&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;13&lt;/span&gt; &lt;span class="mi"&gt;03&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;25&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;00&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;13&lt;/span&gt; &lt;span class="mi"&gt;03&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;42&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;00&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                              &lt;span class="n"&gt;NaN&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;13&lt;/span&gt; &lt;span class="mi"&gt;16&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;12&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;00&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;13&lt;/span&gt; &lt;span class="mi"&gt;16&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;23&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;00&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                              &lt;span class="n"&gt;NaN&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;13&lt;/span&gt; &lt;span class="mi"&gt;15&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;05&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;00&lt;/span&gt;  &lt;span class="mi"&gt;2013&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;01&lt;/span&gt;&lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mi"&gt;13&lt;/span&gt; &lt;span class="mi"&gt;15&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;15&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt;&lt;span class="mi"&gt;00&lt;/span&gt;

                 &lt;span class="n"&gt;passenger_count&lt;/span&gt;  &lt;span class="n"&gt;trip_time_in_secs&lt;/span&gt;  &lt;span class="n"&gt;trip_distance&lt;/span&gt;  \
&lt;span class="n"&gt;passenger_count&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;                              &lt;span class="mi"&gt;0&lt;/span&gt;               &lt;span class="mi"&gt;1020&lt;/span&gt;           &lt;span class="mf"&gt;5.21&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                              &lt;span class="mi"&gt;1&lt;/span&gt;                &lt;span class="mi"&gt;660&lt;/span&gt;           &lt;span class="mf"&gt;2.94&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                              &lt;span class="mi"&gt;1&lt;/span&gt;                &lt;span class="mi"&gt;600&lt;/span&gt;           &lt;span class="mf"&gt;2.18&lt;/span&gt;

                 &lt;span class="n"&gt;pickup_longitude&lt;/span&gt;  &lt;span class="n"&gt;pickup_latitude&lt;/span&gt;  &lt;span class="n"&gt;dropoff_longitude&lt;/span&gt;  \
&lt;span class="n"&gt;passenger_count&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;                      &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.986900&lt;/span&gt;        &lt;span class="mf"&gt;40.743736&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;74.029747&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                      &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.976753&lt;/span&gt;        &lt;span class="mf"&gt;40.790123&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.984802&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                      &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.982719&lt;/span&gt;        &lt;span class="mf"&gt;40.767147&lt;/span&gt;         &lt;span class="o"&gt;-&lt;/span&gt;&lt;span class="mf"&gt;73.982170&lt;/span&gt;

                 &lt;span class="n"&gt;dropoff_latitude&lt;/span&gt;
&lt;span class="n"&gt;passenger_count&lt;/span&gt;
&lt;span class="mi"&gt;0&lt;/span&gt;                       &lt;span class="mf"&gt;40.741348&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                       &lt;span class="mf"&gt;40.758518&lt;/span&gt;
&lt;span class="mi"&gt;1&lt;/span&gt;                       &lt;span class="mf"&gt;40.746170&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;6&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;d2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;blockdivs&lt;/span&gt;  &lt;span class="c1"&gt;# our new partition values&lt;/span&gt;
&lt;span class="n"&gt;Out&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;6&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;6&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&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="n"&gt;d&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;blockdivs&lt;/span&gt;   &lt;span class="c1"&gt;# our original partition values&lt;/span&gt;
&lt;span class="n"&gt;Out&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;7&lt;/span&gt;&lt;span class="p"&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;20000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;30000&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;50000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;60000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;70000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;80000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;90000&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/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 321)&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="some-problems"&gt;
&lt;h1&gt;Some Problems&lt;/h1&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;First, we have to evaluate the dask as we go. Every &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;set_index&lt;/span&gt;&lt;/code&gt; operation (and
hence many groupbys and joins) forces an evaluation. We can no longer, as in
the dask.array case, endlessly compound high-level operations to form more and
more complex graphs and then only evaluate at the end. We need to evaluate as
we go.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Sorting/shuffling is slow. This is for a few reasons including the
serialization of DataFrames and sorting being hard.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;How feasible is it to frequently re-index a large amount of data? When do we
reach the stage of “just use a database”?&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Pandas doesn’t yet release the GIL, so this is all single-core. See post on
&lt;a class="reference internal" href="#/work/2015/03/10/PyData-GIL/"&gt;&lt;span class="xref myst"&gt;PyData and the GIL&lt;/span&gt;&lt;/a&gt;.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;My current solution lacks basic functionality. I’ve skipped
the easy things to first ensure sure that the hard stuff is doable.&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/2015/03/11/Towards-OOC-Frame.md&lt;/span&gt;, line 341)&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="help"&gt;
&lt;h1&gt;Help&lt;/h1&gt;
&lt;p&gt;I know less about tables than about arrays. I’m ignorant of the literature and
common solutions in this field. If anything here looks suspicious then &lt;em&gt;please
speak up&lt;/em&gt;. I could really use your help.&lt;/p&gt;
&lt;p&gt;Additionally the Pandas API is much more complex than NumPy’s. If any
experienced devs out there feel like jumping in and implementing fairly
straightforward Pandas features in a blocked way I’d be obliged.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/03/11/Towards-OOC-Frame/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-03-11T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/02/17/Towards-OOC-Bag/</id>
    <title>Towards Out-of-core ND-Arrays -- Dask + Toolz = Bag</title>
    <updated>2015-02-17T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr&lt;/strong&gt; We use dask to build a parallel Python list.&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/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 14)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="introduction"&gt;

&lt;p&gt;This is the seventh in a sequence of posts constructing an out-of-core nd-array
using NumPy, and dask. You can view these posts here:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2014/12/27/Towards-OOC/"&gt;&lt;span class="doc std std-doc"&gt;Simple task scheduling&lt;/span&gt;&lt;/a&gt;,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2014/12/30/Towards-OOC-Frontend/"&gt;&lt;span class="doc std std-doc"&gt;Frontend usability&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/01/06/Towards-OOC-Scheduling/"&gt;&lt;span class="doc std std-doc"&gt;A multi-threaded scheduler&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/01/14/Towards-OOC-MatMul/"&gt;&lt;span class="doc std std-doc"&gt;Matrix Multiply Benchmark&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/01/16/Towards-OOC-SpillToDisk/"&gt;&lt;span class="doc std std-doc"&gt;Spilling to disk&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/02/13/Towards-OOC-Slicing-and-Stacking/"&gt;&lt;span class="doc std std-doc"&gt;Slicing and Stacking&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Today we take a break from ND-Arrays and show how task scheduling can attack
other collections like the simple &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;list&lt;/span&gt;&lt;/code&gt; of Python objects.&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/2015/02/17/Towards-OOC-Bag.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="unstructured-data"&gt;
&lt;h1&gt;Unstructured Data&lt;/h1&gt;
&lt;p&gt;Often before we have an &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ndarray&lt;/span&gt;&lt;/code&gt; or a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;table/DataFrame&lt;/span&gt;&lt;/code&gt; we have unstructured
data like log files. In this case tuned subsets of the language (e.g.
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numpy&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pandas&lt;/span&gt;&lt;/code&gt;) aren’t sufficient, we need the full Python language.&lt;/p&gt;
&lt;p&gt;My usual approach to the inconveniently large dump of log files is to use
Python &lt;a class="reference external" href="http://toolz.readthedocs.org/en/latest/streaming-analytics.html"&gt;streaming
iterators&lt;/a&gt;
along with &lt;a class="reference external" href="http://toolz.readthedocs.org/en/latest/parallelism.html"&gt;multiprocessing or IPython
Parallel&lt;/a&gt; on a single
large machine. I often write/speak about this workflow when discussing
&lt;a class="reference external" href="http://toolz.readthedocs.org/"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;toolz&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;.&lt;/p&gt;
&lt;p&gt;This workflow grows complex for most users when you introduce many processes.
To resolve this we build our normal tricks into a new &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.Bag&lt;/span&gt;&lt;/code&gt; collection.&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/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 46)&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="bag"&gt;
&lt;h1&gt;Bag&lt;/h1&gt;
&lt;p&gt;In the same way that &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; mimics NumPy operations (e.g. matrix
multiply, slicing), &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.bag&lt;/span&gt;&lt;/code&gt; mimics functional operations like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;map&lt;/span&gt;&lt;/code&gt;,
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;filter&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;reduce&lt;/span&gt;&lt;/code&gt; found in the standard library as well as many of the
streaming functions found in &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;toolz&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Dask array = NumPy + threads&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask bag = Python/Toolz + processes&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/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 56)&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="example"&gt;
&lt;h1&gt;Example&lt;/h1&gt;
&lt;p&gt;Here’s the obligatory wordcount example&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.bag&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Bag&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Bag&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_filenames&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;data/*.txt&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="k"&gt;def&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nf"&gt;stem&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;word&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;&lt;span class="w"&gt;    &lt;/span&gt;&lt;span class="sd"&gt;&amp;quot;&amp;quot;&amp;quot; Stem word to primitive form &amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;    &lt;span class="k"&gt;return&lt;/span&gt; &lt;span class="n"&gt;word&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;lower&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;rstrip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;,.!:;&amp;#39;-&lt;/span&gt;&lt;span class="se"&gt;\&amp;quot;&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;lstrip&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&amp;#39;&lt;/span&gt;&lt;span class="se"&gt;\&amp;quot;&lt;/span&gt;&lt;span class="s2"&gt;&amp;quot;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="nb"&gt;str&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="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;concat&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;stem&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;frequencies&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;span class="go"&gt;{...}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We use all of our cores and stream through memory on each core. We use
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;multiprocessing&lt;/span&gt;&lt;/code&gt; but could get fancier with some work.&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/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 76)&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="related-work"&gt;
&lt;h1&gt;Related Work&lt;/h1&gt;
&lt;p&gt;There are a lot of much larger much more powerful systems that have a similar
API, notably &lt;a class="reference external" href="http://spark.apache.org/"&gt;Spark&lt;/a&gt; and
&lt;a class="reference external" href="https://github.com/douban/dpark"&gt;DPark&lt;/a&gt;. If you have &lt;em&gt;Big Data&lt;/em&gt; and need to
use very many machines then you should stop reading this and go install them.&lt;/p&gt;
&lt;p&gt;I mostly made dask.bag because&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;It was very easy given the work already done on dask.array&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;I often only need multiprocessing + a heavy machine&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;I wanted something trivially pip installable that didn’t use the JVM&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;But again, if you have &lt;em&gt;Big Data&lt;/em&gt;, then this isn’t for you.&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/2015/02/17/Towards-OOC-Bag.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="design"&gt;
&lt;h1&gt;Design&lt;/h1&gt;
&lt;p&gt;As before, a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Bag&lt;/span&gt;&lt;/code&gt; is just a dict holding tasks, along with a little meta data.&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;d&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;{(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&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="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;5&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;     &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="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;5&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;     &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="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;5&lt;/span&gt;&lt;span class="p"&gt;)}&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.bag&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Bag&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Bag&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;x&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;npartitions&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;In this way we break up one collection&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;[0, 1, 2, 3, 4, 0, 1, 2, 3, 4, 0, 1, 2, 3, 4]
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;into three independent pieces&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;[0, 1, 2, 3, 4]
[0, 1, 2, 3, 4]
[0, 1, 2, 3, 4]
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;When we abstractly operate on the large 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;b2&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;… we generate new tasks to operate on each of the components.&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;b2&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;
&lt;span class="go"&gt;{(&amp;#39;x&amp;#39;, 0): (range, 5),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 1): (range, 5),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;x&amp;#39;, 2): (range, 5)}&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;bag-1&amp;#39;, 0): (map, lambda x: x * 10, (&amp;#39;x&amp;#39;, 0)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;bag-1&amp;#39;, 1): (map, lambda x: x * 10, (&amp;#39;x&amp;#39;, 1)),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;bag-1&amp;#39;, 2): (map, lambda x: x * 10, (&amp;#39;x&amp;#39;, 2))}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;And when we ask for concrete results (the call to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;list&lt;/span&gt;&lt;/code&gt;) we spin up a
scheduler to execute the resulting dependency graph of tasks.&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="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;b2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;[0, 10, 20, 30, 40, 0, 10, 20, 30, 40, 0, 10, 20, 30, 40]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;More complex operations yield more complex dasks. Beware, dask code is pretty
Lispy. Fortunately these dasks are internal; users don’t interact with them.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;iseven&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;%&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt; &lt;span class="o"&gt;==&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;b3&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;filter&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;iseven&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;count&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dask&lt;/span&gt;
&lt;span class="go"&gt;{&amp;#39;bag-3&amp;#39;: (sum, [(&amp;#39;bag-2&amp;#39;, 1), (&amp;#39;bag-2&amp;#39;, 2), (&amp;#39;bag-2&amp;#39;, 0)]),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;bag-2&amp;#39;, 0): (count,&lt;/span&gt;
&lt;span class="go"&gt;                (filter, iseven, (range, 5))),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;bag-2&amp;#39;, 1): (count,&lt;/span&gt;
&lt;span class="go"&gt;                (filter, iseven, (range, 5))),&lt;/span&gt;
&lt;span class="go"&gt; (&amp;#39;bag-2&amp;#39;, 2): (count,&lt;/span&gt;
&lt;span class="go"&gt;                (filter, iseven, (range, 5)))}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The current interface for &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;Bag&lt;/span&gt;&lt;/code&gt; has the following operations:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;all             frequencies         min
any             join                product
count           map                 std
filter          map_partitions      sum
fold            max                 topk
foldby          mean                var
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Manipulations of bags create task dependency graphs. We eventually execute
these graphs in parallel.&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/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 167)&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="execution"&gt;
&lt;h1&gt;Execution&lt;/h1&gt;
&lt;p&gt;We repurpose the threaded scheduler we used for arrays to support
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;multiprocessing&lt;/span&gt;&lt;/code&gt; to provide parallelism even on pure Python code. We’re
careful to avoid unnecessary data transfer. None of the operations listed above
requires significant communication. Notably we don’t have any concept of
&lt;em&gt;shuffling&lt;/em&gt; or scatter/gather.&lt;/p&gt;
&lt;p&gt;We &lt;a class="reference external" href="http://trac.mystic.cacr.caltech.edu/project/pathos/wiki/dill"&gt;use dill&lt;/a&gt; to
take care to serialize functions properly and collect/report errors, two issues
that &lt;a class="reference internal" href="#/2013/12/05/Parallelism-and-Serialization/"&gt;&lt;span class="xref myst"&gt;plague naive use of
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;multiprocessing&lt;/span&gt;&lt;/code&gt;&lt;/span&gt;&lt;/a&gt; in Python.&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="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;  &lt;span class="c1"&gt;# This works!&lt;/span&gt;
&lt;span class="go"&gt;[0, 10, 20, 30, 40, 0, 10, 20, 30, 40, 0, 10, 20, 30, 40]&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &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;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;   &lt;span class="c1"&gt;# This errs gracefully!&lt;/span&gt;
&lt;span class="go"&gt;ZeroDivisionError:  Execption in remote Process&lt;/span&gt;

&lt;span class="go"&gt;integer division or modulo by zero&lt;/span&gt;

&lt;span class="go"&gt;Traceback:&lt;/span&gt;
&lt;span class="go"&gt;    ...&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;These tricks remove need for user expertise.&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/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 195)&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="productive-sweet-spot"&gt;
&lt;h1&gt;Productive Sweet Spot&lt;/h1&gt;
&lt;p&gt;I think that there is a productive sweet spot in the following configuration&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Pure Python functions&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Streaming/lazy data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Multiprocessing&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A single large machine or a few machines in an informal cluster&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;This setup is common and it’s capable of handling terabyte scale workflows.
In my brief experience people rarely take this route. They use single-threaded
in-memory Python until it breaks, and then seek out &lt;em&gt;Big Data Infrastructure&lt;/em&gt;
like Hadoop/Spark at relatively high productivity overhead.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Your workstation can scale bigger than you think.&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/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 211)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="id1"&gt;
&lt;h1&gt;Example&lt;/h1&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: INFO/1 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 211); &lt;em&gt;&lt;a href="#id1"&gt;backlink&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;Duplicate implicit target name: “example”.&lt;/p&gt;
&lt;/aside&gt;
&lt;p&gt;Here is about a gigabyte of
&lt;a class="reference external" href="http://ita.ee.lbl.gov/html/contrib/UCB.home-IP-HTTP.html"&gt;network flow data&lt;/a&gt;,
recording which computers made connections to which other computers on the
UC-Berkeley campus in 1996.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;846890339:661920 846890339:755475 846890340:197141 168.237.7.10:1163 83.153.38.208:80 2 8 4294967295 4294967295 846615753 176 2462 39 GET 21068906053917068819..html HTTP/1.0

846890340:989181 846890341:2147 846890341:2268 13.35.251.117:1269 207.83.232.163:80 10 0 842099997 4294967295 4294967295 64 1 38 GET 20271810743860818265..gif HTTP/1.0

846890341:80714 846890341:90331 846890341:90451 13.35.251.117:1270 207.83.232.163:80 10 0 842099995 4294967295 4294967295 64 1 38 GET 38127854093537985420..gif HTTP/1.0
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is actually relatively clean. Many of the fields are space delimited (not
all) and I’ve already compiled and run the decade old C-code needed to
decompress it from its original format.&lt;/p&gt;
&lt;p&gt;Lets use Bag and regular expressions to parse this.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.bag&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt; &lt;span class="n"&gt;Bag&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;into&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Bag&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_filenames&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;UCB-home-IP*.log&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;re&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;pattern&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="s2"&gt;&amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;request_time&amp;gt;\d+:\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;response_start&amp;gt;\d+:\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;response_end&amp;gt;\d+:\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;client_ip&amp;gt;\d+\.\d+\.\d+\.\d+):(?P&amp;lt;client_port&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;server_ip&amp;gt;\d+\.\d+\.\d+\.\d+):(?P&amp;lt;server_port&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;client_headers&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;server_headers&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;if_modified_since&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;response_header_length&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;response_data_length&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;request_url_length&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;expires&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;last_modified&amp;gt;\d+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;method&amp;gt;\w+)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;domain&amp;gt;\d+..)\.(?P&amp;lt;extension&amp;gt;\w*)(?P&amp;lt;rest_of_url&amp;gt;\S*)&lt;/span&gt;
&lt;span class="s2"&gt;   ...: (?P&amp;lt;protocol&amp;gt;.*)&amp;quot;&amp;quot;&amp;quot;&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;strip&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;replace&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;&lt;/span&gt;&lt;span class="se"&gt;\n&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;\s+&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;prog&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;re&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compile&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pattern&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;6&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;records&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;b&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;prog&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;match&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;m&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;m&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupdict&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This returns instantly. We only compute results when necessary. We trigger
computation by calling &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;list&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;In&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="n"&gt;records&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;take&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;Out&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="p"&gt;[{&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;client_headers&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;client_ip&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;168.237.7.10&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;client_port&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;1163&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;domain&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;21068906053917068819.&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;expires&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;2462&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;extension&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;html&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;if_modified_since&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;4294967295&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;last_modified&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;39&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;method&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;GET&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;protocol&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;HTTP/1.0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;request_time&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;846890339:661920&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;request_url_length&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;176&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;response_data_length&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;846615753&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;response_end&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;846890340:197141&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;response_header_length&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;4294967295&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;response_start&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;846890339:755475&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;rest_of_url&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;server_headers&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;server_ip&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;83.153.38.208&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
  &lt;span class="s1"&gt;&amp;#39;server_port&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;80&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;}]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Because bag operates lazily this small result also returns immediately.&lt;/p&gt;
&lt;p&gt;To demonstrate depth we find the ten client/server pairs with the most
connections.&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;8&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;counts&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;records&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;pluck&lt;/span&gt;&lt;span class="p"&gt;([&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;client_ip&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;server_ip&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;frequencies&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;9&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="nb"&gt;list&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;counts&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;topk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;key&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]))&lt;/span&gt;
&lt;span class="n"&gt;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="mf"&gt;11.2&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;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mf"&gt;1.15&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;total&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mf"&gt;12.3&lt;/span&gt; &lt;span class="n"&gt;s&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="mf"&gt;50.4&lt;/span&gt; &lt;span class="n"&gt;s&lt;/span&gt;
&lt;span class="n"&gt;Out&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;9&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
&lt;span class="p"&gt;[((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;247.193.34.56&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;243.182.247.102&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;35353&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
 &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;172.219.28.251&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;47.61.128.1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;22333&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
 &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;240.97.200.0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;108.146.202.184&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;17492&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
 &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;229.112.177.58&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;47.61.128.1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;12993&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
 &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;146.214.34.69&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;119.153.78.6&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;12554&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
 &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;17.32.139.174&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;179.135.20.36&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;10166&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
 &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;97.166.76.88&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;65.81.49.125&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;8155&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
 &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;55.156.159.21&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;157.229.248.255&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;7533&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
 &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;55.156.159.21&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;124.77.75.86&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;7506&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt;
 &lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;55.156.159.21&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;97.5.181.76&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="mi"&gt;7501&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/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 312)&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="comparison-with-spark"&gt;
&lt;h1&gt;Comparison with Spark&lt;/h1&gt;
&lt;p&gt;First, it is silly and unfair to compare with PySpark running locally. PySpark
offers much more in a distributed context.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;pyspark&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;sc&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;pyspark&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;SparkContext&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;local&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;3&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;glob&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;glob&lt;/span&gt;
&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;filenames&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;sorted&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;glob&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;UCB-home-*.log&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;5&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;rdd&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;sc&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;parallelize&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;filenames&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;numSlices&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;re&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;7&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;pattern&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&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;8&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;prog&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;re&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compile&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;pattern&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;9&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;lines&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;rdd&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;flatMap&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;fn&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;open&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;fn&lt;/span&gt;&lt;span class="p"&gt;)))&lt;/span&gt;
&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;records&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;lines&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;line&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="n"&gt;prog&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;match&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;line&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;groupdict&lt;/span&gt;&lt;span class="p"&gt;())&lt;/span&gt;
&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;11&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt; &lt;span class="n"&gt;ips&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;records&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;map&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="k"&gt;lambda&lt;/span&gt; &lt;span class="n"&gt;rec&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;rec&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;client_ip&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;rec&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;server_ip&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]))&lt;/span&gt;

&lt;span class="n"&gt;In&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;12&lt;/span&gt;&lt;span class="p"&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;toolz&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;topk&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;13&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="nb"&gt;dict&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;topk&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;10&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;ips&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;countByValue&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;items&lt;/span&gt;&lt;span class="p"&gt;(),&lt;/span&gt; &lt;span class="n"&gt;key&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;))&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="mf"&gt;1.32&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;sys&lt;/span&gt;&lt;span class="p"&gt;:&lt;/span&gt; &lt;span class="mf"&gt;52.2&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="mf"&gt;1.37&lt;/span&gt; &lt;span class="n"&gt;s&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;1&lt;/span&gt;&lt;span class="nb"&gt;min&lt;/span&gt; &lt;span class="mi"&gt;21&lt;/span&gt;&lt;span class="n"&gt;s&lt;/span&gt;
&lt;span class="n"&gt;Out&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;13&lt;/span&gt;&lt;span class="p"&gt;]:&lt;/span&gt;
&lt;span class="p"&gt;{(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;146.214.34.69&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;119.153.78.6&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;12554&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
 &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;17.32.139.174&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;179.135.20.36&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;10166&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
 &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;172.219.28.251&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;47.61.128.1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;22333&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
 &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;229.112.177.58&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;47.61.128.1&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;12993&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
 &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;240.97.200.0&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;108.146.202.184&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;17492&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
 &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;247.193.34.56&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;243.182.247.102&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;35353&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
 &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;55.156.159.21&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;124.77.75.86&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;7506&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
 &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;55.156.159.21&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;157.229.248.255&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;7533&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
 &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;55.156.159.21&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;97.5.181.76&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;7501&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
 &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;97.166.76.88&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;65.81.49.125&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="mi"&gt;8155&lt;/span&gt;&lt;span class="p"&gt;}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;So, given a compute-bound mostly embarrassingly parallel task (regexes are
comparatively expensive) on a single machine they are comparable.&lt;/p&gt;
&lt;p&gt;Reasons you would want to use Spark&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;You want to use many machines and interact with HDFS&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Shuffling operations&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Reasons you would want to use dask.bag&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Trivial installation&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;No mucking about with JVM heap sizes or config files&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Nice error reporting. Spark error reporting includes the typical giant
Java Stack trace that comes with any JVM solution.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Easier/simpler for Python programmers to hack on.
The implementation is 350 lines including comments.&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Again, this is really just a toy experiment to show that the dask model isn’t
just about arrays. I absolutely do not want to throw Dask in the ring with
Spark.&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/2015/02/17/Towards-OOC-Bag.md&lt;/span&gt;, line 372)&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;However I do want to stress the importance of single-machine parallelism.
Dask.bag targets this application well and leverages common hardware in a
simple way that is both natural and accessible to most Python programmers.&lt;/p&gt;
&lt;p&gt;A skilled developer could extend this to work in a distributed memory context.
The logic to create the task dependency graphs is separate from the scheduler.&lt;/p&gt;
&lt;p&gt;Special thanks to &lt;a class="reference external" href="http://github.com/eriknw"&gt;Erik Welch&lt;/a&gt; for finely crafting
the dask optimization passes that keep the data flowly smoothly.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/02/17/Towards-OOC-Bag/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-02-17T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/02/13/Towards-OOC-Slicing-and-Stacking/</id>
    <title>Towards Out-of-core ND-Arrays -- Slicing and Stacking</title>
    <updated>2015-02-13T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr&lt;/strong&gt; Dask.arrays can slice and stack. This is useful for weather data.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/02/13/Towards-OOC-Slicing-and-Stacking.md&lt;/span&gt;, line 14)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="introduction"&gt;

&lt;p&gt;This is the sixth in a sequence of posts constructing an out-of-core nd-array
using NumPy, and dask. You can view these posts here:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2014/12/27/Towards-OOC/"&gt;&lt;span class="doc std std-doc"&gt;Simple task scheduling&lt;/span&gt;&lt;/a&gt;,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2014/12/30/Towards-OOC-Frontend/"&gt;&lt;span class="doc std std-doc"&gt;Frontend usability&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/01/06/Towards-OOC-Scheduling/"&gt;&lt;span class="doc std std-doc"&gt;A multi-threaded scheduler&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/01/14/Towards-OOC-MatMul/"&gt;&lt;span class="doc std std-doc"&gt;Matrix Multiply Benchmark&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/01/16/Towards-OOC-SpillToDisk/"&gt;&lt;span class="doc std std-doc"&gt;Spilling to disk&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Now we talk about slicing and stacking. We use meteorological data as an
example use case.&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/2015/02/13/Towards-OOC-Slicing-and-Stacking.md&lt;/span&gt;, line 28)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="slicing"&gt;
&lt;h1&gt;Slicing&lt;/h1&gt;
&lt;p&gt;Dask.array now supports most of the NumPy slicing syntax. In particular it
supports the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Slicing by integers and slices &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x[0,&lt;/span&gt; &lt;span class="pre"&gt;:5]&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Slicing by a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;list&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.ndarray&lt;/span&gt;&lt;/code&gt; of integers &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x[[1,&lt;/span&gt; &lt;span class="pre"&gt;2,&lt;/span&gt; &lt;span class="pre"&gt;4]]&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Slicing by a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;list&lt;/span&gt;&lt;/code&gt;/&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.ndarray&lt;/span&gt;&lt;/code&gt; of booleans &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x[[False,&lt;/span&gt; &lt;span class="pre"&gt;True,&lt;/span&gt; &lt;span class="pre"&gt;True,&lt;/span&gt; &lt;span class="pre"&gt;False,&lt;/span&gt; &lt;span class="pre"&gt;True]]&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;It does not currently support the following:&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Slicing one &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; with another &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x[x&lt;/span&gt; &lt;span class="pre"&gt;&amp;gt;&lt;/span&gt; &lt;span class="pre"&gt;0]&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Slicing with lists in multiple axes &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x[[1,&lt;/span&gt; &lt;span class="pre"&gt;2,&lt;/span&gt; &lt;span class="pre"&gt;3],&lt;/span&gt; &lt;span class="pre"&gt;[3,&lt;/span&gt; &lt;span class="pre"&gt;2,&lt;/span&gt; &lt;span class="pre"&gt;1]]&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/02/13/Towards-OOC-Slicing-and-Stacking.md&lt;/span&gt;, line 42)&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="stack-and-concatenate"&gt;
&lt;h1&gt;Stack and Concatenate&lt;/h1&gt;
&lt;p&gt;We often store large arrays on disk in many different files. We
want to stack or concatenate these arrays together into one logical array.
Dask solves this problem with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;stack&lt;/span&gt;&lt;/code&gt; and &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;concatenate&lt;/span&gt;&lt;/code&gt; functions, which
stitch many arrays together into a single array, either creating a new
dimension with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;stack&lt;/span&gt;&lt;/code&gt; or along an existing dimension with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;concatenate&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;section id="stack"&gt;
&lt;h2&gt;Stack&lt;/h2&gt;
&lt;p&gt;We stack many existing dask arrays into a new array, creating a new dimension
as we go.&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.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;from_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ones&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt; &lt;span class="n"&gt;blockshape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="gp"&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="ow"&gt;in&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;3&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;  &lt;span class="c1"&gt;# A small stack of dask arrays&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;stack&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;
&lt;span class="go"&gt;(3, 4, 4)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;stack&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;
&lt;span class="go"&gt;(4, 3, 4)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;stack&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;
&lt;span class="go"&gt;(4, 4, 3)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This creates a new dimension with length equal to the number of slices&lt;/p&gt;
&lt;/section&gt;
&lt;section id="concatenate"&gt;
&lt;h2&gt;Concatenate&lt;/h2&gt;
&lt;p&gt;We concatenate existing arrays into a new array, extending them along an
existing dimension&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.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;from_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ones&lt;/span&gt;&lt;span class="p"&gt;((&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt; &lt;span class="n"&gt;blockshape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="gp"&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="ow"&gt;in&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;3&lt;/span&gt;&lt;span class="p"&gt;)]&lt;/span&gt;  &lt;span class="c1"&gt;# small stack of dask arrays&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;concatenate&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;
&lt;span class="go"&gt;(12, 4)&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;concatenate&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;1&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;
&lt;span class="go"&gt;(4, 12)&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/2015/02/13/Towards-OOC-Slicing-and-Stacking.md&lt;/span&gt;, line 89)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;/section&gt;
&lt;section id="case-study-with-meteorological-data"&gt;
&lt;h1&gt;Case Study with Meteorological Data&lt;/h1&gt;
&lt;p&gt;To test this new functionality we download &lt;a class="reference external" href="http://www.ecmwf.int/en/research/climate-reanalysis/era-interim"&gt;meteorological
data&lt;/a&gt; from the
&lt;a class="reference external" href="http://www.ecmwf.int/#main-menu"&gt;European Centre for Medium-Range Weather
Forecasts&lt;/a&gt;. In particular we have the
temperature for the Earth every six hours for all of 2014 with spatial
resolution of a quarter degree. We download this data using &lt;a class="reference external" href="https://gist.github.com/mrocklin/26d8323f9a8a6a75fce0"&gt;this
script&lt;/a&gt; (please don’t
hammer their servers unnecessarily) (Thanks due to &lt;a class="reference external" href="http://stephanhoyer.com/"&gt;Stephan
Hoyer&lt;/a&gt; for pointing me to this dataset).&lt;/p&gt;
&lt;p&gt;As a result, I now have a bunch of netCDF files!&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;$ ls
2014-01-01.nc3  2014-03-18.nc3  2014-06-02.nc3  2014-08-17.nc3  2014-11-01.nc3
2014-01-02.nc3  2014-03-19.nc3  2014-06-03.nc3  2014-08-18.nc3  2014-11-02.nc3
2014-01-03.nc3  2014-03-20.nc3  2014-06-04.nc3  2014-08-19.nc3  2014-11-03.nc3
2014-01-04.nc3  2014-03-21.nc3  2014-06-05.nc3  2014-08-20.nc3  2014-11-04.nc3
...             ...             ...             ...             ...
&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;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;netCDF4&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;t&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;netCDF4&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="s1"&gt;&amp;#39;2014-01-01.nc3&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;variables&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;t2m&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;t&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;
&lt;span class="go"&gt;(4, 721, 1440)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The shape corresponds to four measurements per day (24h / 6h), 720 measurements
North/South (180 / 0.25) and 1440 measurements East/West (360/0.25). There are
365 files.&lt;/p&gt;
&lt;p&gt;Great! We collect these under one logical dask array, concatenating along
the time axis.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;glob&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;glob&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;filenames&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="nb"&gt;sorted&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;glob&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;2014-*.nc3&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;temps&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;netCDF4&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;fn&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;variables&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;t2m&amp;#39;&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;fn&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;filenames&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="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.array&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;da&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="n"&gt;da&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;from_array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;t&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;blockshape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;200&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;200&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;t&lt;/span&gt; &lt;span class="ow"&gt;in&lt;/span&gt; &lt;span class="n"&gt;temps&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;x&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;concatenate&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;arrays&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;0&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;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;shape&lt;/span&gt;
&lt;span class="go"&gt;(1464, 721, 1440)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now we can play with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x&lt;/span&gt;&lt;/code&gt; as though it were a NumPy array.&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;avg&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;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;diff&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;x&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="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;avg&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;If we want to actually compute these results we have a few options&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;diff&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;compute&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;  &lt;span class="c1"&gt;# compute result, return as array, float, int, whatever is appropriate&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;array&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;diff&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# compute result and turn into `np.ndarray`&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;diff&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;store&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;anything_that_supports_setitem&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# For out-of-core storage&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Alternatively, because many scientific Python libraries call &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.array&lt;/span&gt;&lt;/code&gt; on
inputs, we can just feed our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.Array&lt;/span&gt;&lt;/code&gt; objects directly in to matplotlib
(hooray for the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;__array__&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="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;matplotlib&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;imshow&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;cmap&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;bone&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;]&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;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;cmap&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;RdBu_r&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;table&gt;
&lt;tr&gt;
&lt;td&gt;&lt;a href="/images/avg.png"&gt;
        &lt;img src="/images/avg.png" width="100%"&gt;&lt;/a&gt;&lt;/td&gt;
&lt;td&gt;&lt;a href="/images/diff.png"&gt;
        &lt;img src="/images/diff.png" width="100%"&gt;&lt;/a&gt;&lt;/td&gt;
&lt;/tr&gt;
&lt;/table&gt;
&lt;p&gt;I suspect that the temperature scale is in Kelvin. It looks like the random
day is taken during Northern Summer. Another fun one, lets look at the
difference between the temperatures at 00:00 and at 12:00&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;imshow&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[::&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt; &lt;span class="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;x&lt;/span&gt;&lt;span class="p"&gt;[&lt;/span&gt;&lt;span class="mi"&gt;2&lt;/span&gt;&lt;span class="p"&gt;::&lt;/span&gt;&lt;span class="mi"&gt;4&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;cmap&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;RdBu_r&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;a href="/images/day-vs-night.png"&gt;
    &lt;img src="/images/day-vs-night.png" width="100%"&gt;&lt;/a&gt;
&lt;p&gt;Even though this looks and feels like NumPy we’re actually operating off of
disk using blocked algorithms. We execute these operations using only a small
amount of memory. If these operations were computationally intense (they
aren’t) then we also would also benefit from multiple cores.&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/2015/02/13/Towards-OOC-Slicing-and-Stacking.md&lt;/span&gt;, line 187)&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-just-happened"&gt;
&lt;h1&gt;What just happened&lt;/h1&gt;
&lt;p&gt;To be totally clear the following steps just occurred:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Open up a bunch of netCDF files and located a temperature variable
within each file. This is cheap.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;For each of those temperature variables create a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.Array&lt;/span&gt;&lt;/code&gt; object,
specifying how we want to block up that variable. This is also cheap.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Make a new &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.Array&lt;/span&gt;&lt;/code&gt; by concatenating all of our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.Array&lt;/span&gt;&lt;/code&gt;s for each
day. This, like the other steps, is just book-keeping. We haven’t loaded
data or computed anything yet.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Write numpy-style code &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;x[::2].mean(axis=0)&lt;/span&gt; &lt;span class="pre"&gt;-&lt;/span&gt; &lt;span class="pre"&gt;x[2::2].mean(axis=0)&lt;/span&gt;&lt;/code&gt;.
This creates yet another &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.Array&lt;/span&gt;&lt;/code&gt; with a more complex task graph. It
takes a few hundred milliseconds to create this dictionary.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Call&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;imshow&lt;/span&gt;&lt;/code&gt; on our &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;da.Array&lt;/span&gt;&lt;/code&gt; object&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;imshow&lt;/span&gt;&lt;/code&gt; calls &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.array&lt;/span&gt;&lt;/code&gt; on its input, this starts the multi-core task
scheduler&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A flurry of chunks fly out of all the netCDF files. These chunks meet
various NumPy functions and create new chunks. Well organized magic occurs
and an &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.ndarray&lt;/span&gt;&lt;/code&gt; emerges.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Matplotlib makes a pretty graph&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/02/13/Towards-OOC-Slicing-and-Stacking.md&lt;/span&gt;, line 209)&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="problems-that-popped-up"&gt;
&lt;h1&gt;Problems that Popped Up&lt;/h1&gt;
&lt;p&gt;The threaded scheduler is introducing significant overhead in its planning.
For this workflow the single-threaded naive scheduler is actually significantly
faster. We’ll have to find better solutions to reduce scheduling overhead.&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/2015/02/13/Towards-OOC-Slicing-and-Stacking.md&lt;/span&gt;, line 215)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="conclusion"&gt;
&lt;h1&gt;Conclusion&lt;/h1&gt;
&lt;p&gt;I hope that this shows off how &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; can be useful when dealing with
collections of on-disk arrays. As always I’m very happy to hear how we can
make this project more useful for your work. If you have large n-dimensional
datasets I’d love to hear about what you do and how &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; can help. I
can be reached either in the comments below or at &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;blaze-dev&amp;#64;continuum.io&lt;/span&gt;&lt;/code&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/2015/02/13/Towards-OOC-Slicing-and-Stacking.md&lt;/span&gt;, line 223)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="acknowledgements"&gt;
&lt;h1&gt;Acknowledgements&lt;/h1&gt;
&lt;p&gt;First, other projects can already do this. In particular if this seemed useful
for your work then you should probably also know about
&lt;a class="reference external" href="https://github.com/SciTools/biggus"&gt;Biggus&lt;/a&gt;,
produced by the UK Met office, which has been around for much longer than
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask.array&lt;/span&gt;&lt;/code&gt; and is used in production.&lt;/p&gt;
&lt;p&gt;Second, this post shows off work from the following people:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/eriknw"&gt;Erik Welch&lt;/a&gt; (&lt;a class="reference external" href="http://continuum.io"&gt;Continuum&lt;/a&gt;)
wrote optimization passes to clean up dask graphs before execution.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/nevermindewe"&gt;Wesley Emeneker&lt;/a&gt; (&lt;a class="reference external" href="http://continuum.io"&gt;Continuum&lt;/a&gt;) wrote a good deal of the slicing code&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://stephanhoyer.com/"&gt;Stephan Hoyer&lt;/a&gt; (&lt;a class="reference external" href="http://climate.com/"&gt;Climate Corp&lt;/a&gt;)
talked me through the application and pointed me to the data. If you’d
like to see dask integrated with
&lt;a class="reference external" href="http://xray.readthedocs.org/en/stable/"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;xray&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;
then you should definitely bug Stephan :)&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/02/13/Towards-OOC-Slicing-and-Stacking/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-02-13T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/01/16/Towards-OOC-SpillToDisk/</id>
    <title>Towards Out-of-core ND-Arrays -- Spilling to Disk</title>
    <updated>2015-01-16T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr&lt;/strong&gt; We implement a dictionary that spills to disk when we run out of
memory. We connect this to our scheduler.&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/2015/01/16/Towards-OOC-SpillToDisk.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="introduction"&gt;

&lt;p&gt;This is the fifth in a sequence of posts constructing an out-of-core nd-array
using NumPy, Blaze, and dask. You can view these posts here:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2014/12/27/Towards-OOC/"&gt;&lt;span class="doc std std-doc"&gt;Simple task scheduling&lt;/span&gt;&lt;/a&gt;,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2014/12/30/Towards-OOC-Frontend/"&gt;&lt;span class="doc std std-doc"&gt;Frontend usability&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/01/06/Towards-OOC-Scheduling/"&gt;&lt;span class="doc std std-doc"&gt;A multi-threaded scheduler&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/01/14/Towards-OOC-MatMul/"&gt;&lt;span class="doc std std-doc"&gt;Matrix Multiply Benchmark&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We now present &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chest&lt;/span&gt;&lt;/code&gt; a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dict&lt;/span&gt;&lt;/code&gt; type that spills to disk when we run out of
memory. We show how it prevents large computations from flooding memory.&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/2015/01/16/Towards-OOC-SpillToDisk.md&lt;/span&gt;, line 28)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="intermediate-data"&gt;
&lt;h1&gt;Intermediate Data&lt;/h1&gt;
&lt;p&gt;&lt;img src="/images/dask/fail-case.gif"
      align="right"
      width="50%"
      alt="A case where our scheduling algorithm fails to avoid intermediates"&gt;&lt;/p&gt;
&lt;p&gt;If you read the
&lt;a class="reference internal" href="../../2015/01/06/Towards-OOC-Scheduling/"&gt;&lt;span class="doc std std-doc"&gt;post on scheduling&lt;/span&gt;&lt;/a&gt;
you may recall our goal to minimize intermediate storage during a multi-worker
computation. The image on the right shows a trace of our scheduler as it
traverses a task dependency graph. We want to compute the entire graph quickly
while keeping only a small amount of data in memory at once.&lt;/p&gt;
&lt;p&gt;Sometimes we fail and our scheduler stores many large intermediate results. In
these cases we want to spill excess intermediate data to disk rather than
flooding local memory.&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/2015/01/16/Towards-OOC-SpillToDisk.md&lt;/span&gt;, line 46)&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="chest"&gt;
&lt;h1&gt;Chest&lt;/h1&gt;
&lt;p&gt;Chest is a dict-like object that writes data to disk once it runs out of
memory.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;chest&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;Chest&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;d&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Chest&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;available_memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1e9&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# Only use a GigaByte&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;It satisfies the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;MutableMapping&lt;/span&gt;&lt;/code&gt; interface so it looks and feels exactly like
a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dict&lt;/span&gt;&lt;/code&gt;. Below we show an example using a chest with only enough data to
store one Python integer in memory.&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;d&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Chest&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;available_memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;24&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;  &lt;span class="c1"&gt;# enough space for one Python integer&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;one&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;1&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;two&amp;#39;&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="gp"&gt;&amp;gt;&amp;gt;&amp;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;three&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;three&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;]&lt;/span&gt;
&lt;span class="go"&gt;3&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;keys&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;span class="go"&gt;[&amp;#39;one&amp;#39;, &amp;#39;two&amp;#39;, &amp;#39;three&amp;#39;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We keep some data in memory&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;d&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;inmem&lt;/span&gt;
&lt;span class="go"&gt;{&amp;#39;three&amp;#39;: 3}&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;While the rest lives on disk&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;d&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;
&lt;span class="go"&gt;&amp;#39;/tmp/tmpb5ouAb.chest&amp;#39;&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;listdir&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;d&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;path&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="go"&gt;[&amp;#39;one&amp;#39;, &amp;#39;two&amp;#39;]&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;By default we store data with pickle but &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chest&lt;/span&gt;&lt;/code&gt; supports any protocol
with the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dump/load&lt;/span&gt;&lt;/code&gt; interface (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pickle&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;json&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;cbor&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;joblib&lt;/span&gt;&lt;/code&gt;, ….)&lt;/p&gt;
&lt;p&gt;A quick point about &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pickle&lt;/span&gt;&lt;/code&gt;. Frequent readers of my blog may know of my
sadness at how this library
&lt;a class="reference internal" href="#/2013/12/05/Parallelism-and-Serialization/"&gt;&lt;span class="xref myst"&gt;serializes functions&lt;/span&gt;&lt;/a&gt;
and the crippling effect on multiprocessing.
That sadness does not extend to normal data. Pickle is fine for data if you
use the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;protocol=&lt;/span&gt;&lt;/code&gt; keyword to &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;pickle.dump&lt;/span&gt;&lt;/code&gt; correctly . Pickle isn’t a good
cross-language solution, but that doesn’t matter in our application of
temporarily storing numpy arrays on disk.&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/2015/01/16/Towards-OOC-SpillToDisk.md&lt;/span&gt;, line 100)&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="recent-tweaks"&gt;
&lt;h1&gt;Recent tweaks&lt;/h1&gt;
&lt;p&gt;In using &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chest&lt;/span&gt;&lt;/code&gt; alongside &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt; with any reasonable success I had to make the
following improvements to the original implementation:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;A basic LRU mechanism to write only infrequently used data&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;A policy to avoid writing the same data to disk twice if it hasn’t changed&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Thread safety&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Now we can execute more dask workflows without risk of flooding memory&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="o"&gt;...&lt;/span&gt;
&lt;span class="n"&gt;B&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="o"&gt;...&lt;/span&gt;
&lt;span class="n"&gt;expr&lt;/span&gt; &lt;span class="o"&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;T&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dot&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="o"&gt;-&lt;/span&gt; &lt;span class="n"&gt;B&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;mean&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;axis&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;cache&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Chest&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;available_memory&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1e9&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;into&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.hdf5::/result&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;expr&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;cache&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="n"&gt;cache&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Now we incur only moderate slowdown when we schedule poorly and run into large
quantities of intermediate data.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/01/16/Towards-OOC-SpillToDisk.md&lt;/span&gt;, line 124)&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;Chest is only useful when we fail to schedule well. We can still improve
scheduling algorithms to avoid keeping data in memory but it’s nice to have
&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chest&lt;/span&gt;&lt;/code&gt; as a backup for when these algorithms fail. Resilience is reassuring.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/01/16/Towards-OOC-SpillToDisk/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-01-16T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/01/14/Towards-OOC-MatMul/</id>
    <title>Towards Out-of-core ND-Arrays -- Benchmark MatMul</title>
    <updated>2015-01-14T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr&lt;/strong&gt; We benchmark dask on an out-of-core dot product. We also compare and
motivate the use of an optimized BLAS.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Here are the results&lt;/strong&gt;&lt;/p&gt;
&lt;table&gt;
  &lt;thead&gt;
  &lt;tr&gt;
    &lt;th&gt;Performance (GFLOPS)&lt;/th&gt;
    &lt;th&gt;In-Memory&lt;/th&gt;
    &lt;th&gt;On-Disk&lt;/th&gt;
  &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;Reference BLAS&lt;/th&gt;
      &lt;td&gt;6&lt;/td&gt;
      &lt;td&gt;18&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;OpenBLAS one thread&lt;/th&gt;
      &lt;td&gt;11&lt;/td&gt;
      &lt;td&gt;23&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;OpenBLAS four threads&lt;/th&gt;
      &lt;td&gt;22&lt;/td&gt;
      &lt;td&gt;11&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;&lt;em&gt;Disclaimer: This post is on experimental buggy code. This is not ready for public use.&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/2015/01/14/Towards-OOC-MatMul.md&lt;/span&gt;, line 46)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="introduction"&gt;

&lt;p&gt;This is the fourth in a sequence of posts constructing an out-of-core nd-array
using NumPy, Blaze, and dask. You can view these posts here:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2014/12/27/Towards-OOC/"&gt;&lt;span class="doc std std-doc"&gt;Simple task scheduling&lt;/span&gt;&lt;/a&gt;,&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2014/12/30/Towards-OOC-Frontend/"&gt;&lt;span class="doc std std-doc"&gt;Frontend usability&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference internal" href="../../2015/01/06/Towards-OOC-Scheduling/"&gt;&lt;span class="doc std std-doc"&gt;A multi-threaded scheduler&lt;/span&gt;&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We now give performance numbers on out-of-core matrix-matrix multiplication.&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/2015/01/14/Towards-OOC-MatMul.md&lt;/span&gt;, line 57)&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="matrix-matrix-multiplication"&gt;
&lt;h1&gt;Matrix-Matrix Multiplication&lt;/h1&gt;
&lt;p&gt;Dense matrix-matrix multiplication is compute-bound, not I/O bound.
We spend most of our time doing arithmetic and relatively little time shuffling
data around. As a result we may be able to read &lt;em&gt;large&lt;/em&gt; data from disk without
performance loss.&lt;/p&gt;
&lt;p&gt;When multiplying two &lt;span class="math notranslate nohighlight"&gt;\(n\times n\)&lt;/span&gt; matrices we read &lt;span class="math notranslate nohighlight"&gt;\(n^2\)&lt;/span&gt; bytes but perform &lt;span class="math notranslate nohighlight"&gt;\(n^3\)&lt;/span&gt;
computations. There are &lt;span class="math notranslate nohighlight"&gt;\(n\)&lt;/span&gt; computations to do per byte so, relatively
speaking, I/O is cheap.&lt;/p&gt;
&lt;p&gt;We normally measure speed for single CPUs in Giga Floating Point Operations
Per Second (GFLOPS). Lets look at how my laptop does on single-threaded
in-memory matrix-matrix multiplication using NumPy.&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ones&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="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;f8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;timeit&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;dot&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;# Matrix-matrix multiplication&lt;/span&gt;
&lt;span class="go"&gt;10 loops, best of 3: 176 ms per loop&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="mi"&gt;1000&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mf"&gt;0.176&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mf"&gt;1e9&lt;/span&gt;  &lt;span class="c1"&gt;# n cubed computations / seconds / billion&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="mf"&gt;5.681818181818182&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;OK, so NumPy’s matrix-matrix multiplication clocks in at 6 GFLOPS more or
less. The &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.dot&lt;/span&gt;&lt;/code&gt; function ties in to the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;GEMM&lt;/span&gt;&lt;/code&gt; operation in the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;BLAS&lt;/span&gt;&lt;/code&gt;
library on my machine. Currently my &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;numpy&lt;/span&gt;&lt;/code&gt; just uses reference BLAS. (you can
check this with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.show_config()&lt;/span&gt;&lt;/code&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/2015/01/14/Towards-OOC-MatMul.md&lt;/span&gt;, line 87)&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="matrix-matrix-multiply-from-disk"&gt;
&lt;h1&gt;Matrix-Matrix Multiply From Disk&lt;/h1&gt;
&lt;p&gt;For matrices too large to fit in memory we compute the solution one part at a
time, loading blocks from disk when necessary. We parallelize this with
multiple threads. Our last post demonstrates how NumPy+Blaze+Dask automates
this for us.&lt;/p&gt;
&lt;p&gt;We perform a simple numerical experiment, using HDF5 as our on-disk store.&lt;/p&gt;
&lt;p&gt;We install stuff&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;conda&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;-c&lt;span class="w"&gt; &lt;/span&gt;blaze&lt;span class="w"&gt; &lt;/span&gt;blaze
pip&lt;span class="w"&gt; &lt;/span&gt;install&lt;span class="w"&gt; &lt;/span&gt;git+https://github.com/ContinuumIO/dask
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We set up a fake dataset on disk&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;h5py&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;f&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;h5py&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;File&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.hdf5&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;A&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;create_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;A&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;200000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;4000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;f8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;                               &lt;span class="n"&gt;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;250&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;250&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;fillvalue&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1.0&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;B&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;create_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;B&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;4000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;4000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;f8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt;
&lt;span class="gp"&gt;... &lt;/span&gt;                               &lt;span class="n"&gt;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;250&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;250&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;fillvalue&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1.0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We tell Dask+Blaze how to interact with that dataset&lt;/p&gt;
&lt;div class="highlight-python notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="kn"&gt;from&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;dask.obj&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;Array&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;blaze&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;Data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;into&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;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;into&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Array&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;myfile.hdf5::/A&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;blockshape&lt;/span&gt;&lt;span class="o"&gt;=&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="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;))&lt;/span&gt;  &lt;span class="c1"&gt;# dask things&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;into&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Array&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;myfile.hdf5::/B&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;blockshape&lt;/span&gt;&lt;span class="o"&gt;=&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="mi"&gt;1000&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;A&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Data&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="c1"&gt;# Blaze things&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;B&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Data&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We compute our desired result, storing back onto disk&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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;time&lt;/span&gt; &lt;span class="n"&gt;into&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.hdf5::/result&amp;#39;&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="n"&gt;dot&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="go"&gt;2min 49s&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="mi"&gt;200000&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;4000&lt;/span&gt; &lt;span class="o"&gt;*&lt;/span&gt; &lt;span class="mi"&gt;4000&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mi"&gt;169&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mf"&gt;1e9&lt;/span&gt;
&lt;span class="go"&gt;18.934911242&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;18.9 GFLOPS, roughly 3 times faster than the in-memory solution. At first
glance this is confusing - shouldn’t we be slower coming from disk? Our
speedup is due to our use of four cores in parallel. This is good, we don’t
experience much slowdown coming from disk.&lt;/p&gt;
&lt;p&gt;It’s as if all of our hard drive just became memory.&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/2015/01/14/Towards-OOC-MatMul.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="openblas"&gt;
&lt;h1&gt;OpenBLAS&lt;/h1&gt;
&lt;p&gt;Reference BLAS is slow; it was written long ago. OpenBLAS is a modern
implementation. I installed OpenBLAS with my system installer (&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;apt-get&lt;/span&gt;&lt;/code&gt;) and
then reconfigured and rebuilt numpy. OpenBLAS supports many cores. We’ll show
timings with one and with four threads.&lt;/p&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nb"&gt;export&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nv"&gt;OPENBLAS_NUM_THREADS&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="m"&gt;1&lt;/span&gt;
ipython
&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;import&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;numpy&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="k"&gt;as&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nn"&gt;np&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="n"&gt;x&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;ones&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="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;f8&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;timeit&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;dot&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="go"&gt;10 loops, best of 3: 89.8 ms per loop&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="mi"&gt;1000&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mf"&gt;0.0898&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mf"&gt;1e9&lt;/span&gt;  &lt;span class="c1"&gt;# compute GFLOPS&lt;/span&gt;
&lt;span class="go"&gt;11.135857461024498&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;div class="highlight-bash notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;&lt;span class="nb"&gt;export&lt;/span&gt;&lt;span class="w"&gt; &lt;/span&gt;&lt;span class="nv"&gt;OPENBLAS_NUM_THREADS&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="m"&gt;4&lt;/span&gt;
ipython
&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="o"&gt;%&lt;/span&gt;&lt;span class="n"&gt;timeit&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;dot&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="go"&gt;10 loops, best of 3: 46.3 ms per loop&lt;/span&gt;

&lt;span class="gp"&gt;&amp;gt;&amp;gt;&amp;gt; &lt;/span&gt;&lt;span class="mi"&gt;1000&lt;/span&gt; &lt;span class="o"&gt;**&lt;/span&gt; &lt;span class="mi"&gt;3&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mf"&gt;0.0463&lt;/span&gt; &lt;span class="o"&gt;/&lt;/span&gt; &lt;span class="mf"&gt;1e9&lt;/span&gt;  &lt;span class="c1"&gt;# compute GFLOPS&lt;/span&gt;
&lt;span class="go"&gt;21.598272138228943&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This is about four times faster than reference. If you’re not already
parallelizing in some other way (like with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dask&lt;/span&gt;&lt;/code&gt;) then you should use a modern
BLAS like OpenBLAS or MKL.&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/2015/01/14/Towards-OOC-MatMul.md&lt;/span&gt;, line 182)&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="openblas-dask"&gt;
&lt;h1&gt;OpenBLAS + dask&lt;/h1&gt;
&lt;p&gt;Finally we run on-disk our experiment again, now with OpenBLAS. We do this
both with OpenBLAS running with one thread and with many threads.&lt;/p&gt;
&lt;p&gt;We’ll skip the code (it’s identical to what’s above) and give a comprehensive
table of results below.&lt;/p&gt;
&lt;p&gt;Sadly the out-of-core solution doesn’t improve much by using OpenBLAS.
Acutally when both OpenBLAS and dask try to parallelize we &lt;em&gt;lose&lt;/em&gt; performance.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/01/14/Towards-OOC-MatMul.md&lt;/span&gt;, line 193)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="results"&gt;
&lt;h1&gt;Results&lt;/h1&gt;
&lt;table&gt;
  &lt;thead&gt;
  &lt;tr&gt;
    &lt;th&gt;Performance (GFLOPS)&lt;/th&gt;
    &lt;th&gt;In-Memory&lt;/th&gt;
    &lt;th&gt;On-Disk&lt;/th&gt;
  &lt;/tr&gt;
  &lt;/thead&gt;
  &lt;tbody&gt;
    &lt;tr&gt;
      &lt;th&gt;Reference BLAS&lt;/th&gt;
      &lt;td&gt;6&lt;/td&gt;
      &lt;td&gt;18&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;OpenBLAS one thread&lt;/th&gt;
      &lt;td&gt;11&lt;/td&gt;
      &lt;td&gt;23&lt;/td&gt;
    &lt;/tr&gt;
    &lt;tr&gt;
      &lt;th&gt;OpenBLAS four threads&lt;/th&gt;
      &lt;td&gt;22&lt;/td&gt;
      &lt;td&gt;11&lt;/td&gt;
    &lt;/tr&gt;
  &lt;/tbody&gt;
&lt;/table&gt;
&lt;p&gt;&lt;strong&gt;tl:dr&lt;/strong&gt; When doing compute intensive work, don’t worry about using disk, just
don’t use two mechisms of parallelism at the same time.&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/2015/01/14/Towards-OOC-MatMul.md&lt;/span&gt;, line 225)&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="main-take-aways"&gt;
&lt;h1&gt;Main Take-Aways&lt;/h1&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;We don’t lose much by operating from disk in compute-intensive tasks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Actually we can improve performance when an optimized BLAS isn’t avaiable.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dask doesn’t benefit much from an optimized BLAS. This is sad and surprising. I expected performance to scale with single-core in-memory performance. Perhaps this is indicative of some other limiting factor&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;One shouldn’t extrapolate too far with these numbers. They’re only relevant for highly compute-bound operations like matrix-matrix multiply&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;Also, thanks to Wesley Emeneker for finding where we were leaking memory,
making results like these possible.&lt;/p&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/01/14/Towards-OOC-MatMul/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-01-14T00:00:00+00:00</published>
  </entry>
  <entry>
    <id>https://blog.dask.org/2015/01/06/Towards-OOC-Scheduling/</id>
    <title>Towards Out-of-core ND-Arrays -- Multi-core Scheduling</title>
    <updated>2015-01-06T00:00:00+00:00</updated>
    <content type="html">&lt;p&gt;&lt;em&gt;This work is supported by &lt;a class="reference external" href="http://continuum.io"&gt;Continuum Analytics&lt;/a&gt;
and the &lt;a class="reference external" href="http://www.darpa.mil/program/XDATA"&gt;XDATA Program&lt;/a&gt;
as part of the &lt;a class="reference external" href="http://blaze.pydata.org"&gt;Blaze Project&lt;/a&gt;&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;tl;dr&lt;/strong&gt; We show off a multi-threaded shared-memory task scheduler. We share
two techniques for space-constrained computing. We end with pretty GIFs.&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Disclaimer: This post is on experimental buggy code. This is not ready for public use.&lt;/em&gt;&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Disclaimer 2: This post is technical and intended for people who care about
task scheduling, not for traditional users.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 20)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;section id="setup"&gt;

&lt;p&gt;My last two posts
(&lt;a class="reference internal" href="../../2014/12/27/Towards-OOC/"&gt;&lt;span class="doc std std-doc"&gt;post 1&lt;/span&gt;&lt;/a&gt;,
&lt;a class="reference internal" href="../../2014/12/30/Towards-OOC-Frontend/"&gt;&lt;span class="doc std std-doc"&gt;post 2&lt;/span&gt;&lt;/a&gt;)
construct an ND-Array library out of a simple task scheduler, NumPy, and Blaze.&lt;/p&gt;
&lt;p&gt;In this post we discuss a more sophisticated scheduler.
In this post we outline a less elegent but more effective scheduler that uses
multiple threads and caching to achieve performance on an interesting class of
array operations.&lt;/p&gt;
&lt;p&gt;We create scheduling policies to minimize the memory footprint of our
computation.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 35)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="example"&gt;
&lt;h1&gt;Example&lt;/h1&gt;
&lt;p&gt;First, we establish value by doing a hard thing well. Given two large
arrays stored in HDF5:&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;h5py&lt;/span&gt;
&lt;span class="n"&gt;f&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;h5py&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;File&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.hdf5&amp;#39;&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="n"&gt;f&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;create_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;A&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;4000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;2000000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;f8&amp;#39;&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;250&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;250&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;fillvalue&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1.0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;B&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;create_dataset&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;B&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;shape&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;4000&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;4000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;dtype&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;f8&amp;#39;&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;250&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;250&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;fillvalue&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="mf"&gt;1.0&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;f&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;close&lt;/span&gt;&lt;span class="p"&gt;()&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;We do a transpose and dot product.&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;blaze&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;Data&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;into&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.obj&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;Array&lt;/span&gt;

&lt;span class="n"&gt;f&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;h5py&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;File&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.hdf5&amp;#39;&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="n"&gt;into&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Array&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;span class="s1"&gt;&amp;#39;A&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;blockshape&lt;/span&gt;&lt;span class="o"&gt;=&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="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;A&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;span class="n"&gt;b&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;into&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;Array&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;span class="s1"&gt;&amp;#39;B&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;],&lt;/span&gt; &lt;span class="n"&gt;blockshape&lt;/span&gt;&lt;span class="o"&gt;=&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="mi"&gt;1000&lt;/span&gt;&lt;span class="p"&gt;),&lt;/span&gt; &lt;span class="n"&gt;name&lt;/span&gt;&lt;span class="o"&gt;=&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;B&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;

&lt;span class="n"&gt;A&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Data&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="n"&gt;B&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;Data&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="n"&gt;expr&lt;/span&gt; &lt;span class="o"&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;T&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;dot&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="n"&gt;result&lt;/span&gt; &lt;span class="o"&gt;=&lt;/span&gt; &lt;span class="n"&gt;into&lt;/span&gt;&lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;myfile.hdf5::/C&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="n"&gt;expr&lt;/span&gt;&lt;span class="p"&gt;)&lt;/span&gt;
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This uses all of our cores and can be done with only 100MB or so of ram. This
is impressive because neither the inputs, outputs, nor any intermediate stage
of the computation can fit in memory.&lt;/p&gt;
&lt;p&gt;We failed to achieve this exactly (see note at bottom) but still, &lt;em&gt;in theory&lt;/em&gt;,
we’re great!&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 76)&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="avoid-intermediates"&gt;
&lt;h1&gt;Avoid Intermediates&lt;/h1&gt;
&lt;p&gt;To keep a small memory footprint we avoid holding on to unnecessary
intermediate data. The full computation graph of a smaller problem
might look like the following:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/dask/uninlined.png"
     alt="Un-inlined dask"&gt;&lt;/p&gt;
&lt;p&gt;Boxes represent data, circles represent functions that run on that data ,
arrows specify which functions produce/consume which data.&lt;/p&gt;
&lt;p&gt;The top row of circles represent the actual blocked dot products (note the many
data dependence arrows originating from them). The bottom row of circles
represents pulling blocks of data from the the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;A&lt;/span&gt;&lt;/code&gt; HDF5 dataset to in-memory
numpy arrays. The second row transposes the blocks from the first row and adds
more blocks from &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;B&lt;/span&gt;&lt;/code&gt;.&lt;/p&gt;
&lt;p&gt;Naively performed, this graph can be very bad; we replicate our data four times
here, once in each of the rows. We pull out all of the chunks, transpose each
of them, and then finally do a dot product. If we couldn’t fit the original
data in memory then there is no way that this will work.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 99)&lt;/p&gt;
&lt;p&gt;Document headings start at H2, not H1 [myst.header]&lt;/p&gt;
&lt;/aside&gt;
&lt;/section&gt;
&lt;section id="function-inlining"&gt;
&lt;h1&gt;Function Inlining&lt;/h1&gt;
&lt;p&gt;We resolve this in two ways. First, we don’t cache intermediate values for
fast-running functions (like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.transpose&lt;/span&gt;&lt;/code&gt;). Instead we inline fast functions
into tasks involving expensive functions (like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.dot&lt;/span&gt;&lt;/code&gt;).
We may end up running the same quick function twice, but at least we
won’t have to store the result. We trade computation for memory.&lt;/p&gt;
&lt;p&gt;The result of the graph above with all access and transpose operations
inlined looks like the following:&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/dask/inlined.png"
     alt="inlined dask"&gt;&lt;/p&gt;
&lt;p&gt;Now our tasks nest (see below). We run all functions within a nested task as
a single operation. (non-LISP-ers avert your eyes):&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="p"&gt;(&lt;/span&gt;&lt;span class="s1"&gt;&amp;#39;x_6&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;6&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="mi"&gt;0&lt;/span&gt;&lt;span class="p"&gt;):&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;dotmany&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;transpose&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ndslice&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;A&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="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="mi"&gt;1000&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;6&lt;/span&gt;&lt;span class="p"&gt;)),&lt;/span&gt;
                          &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;np&lt;/span&gt;&lt;span class="o"&gt;.&lt;/span&gt;&lt;span class="n"&gt;transpose&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ndslice&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;A&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="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="mi"&gt;1000&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;6&lt;/span&gt;&lt;span class="p"&gt;))],&lt;/span&gt;
                          &lt;span class="p"&gt;[(&lt;/span&gt;&lt;span class="n"&gt;ndslice&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;B&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1000&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="mi"&gt;0&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="p"&gt;(&lt;/span&gt;&lt;span class="n"&gt;ndslice&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="s1"&gt;&amp;#39;B&amp;#39;&lt;/span&gt;&lt;span class="p"&gt;,&lt;/span&gt; &lt;span class="p"&gt;(&lt;/span&gt;&lt;span class="mi"&gt;1000&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="mi"&gt;1&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;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This effectively shoves all of the storage responsibility back down to the HDF5
store. We end up pulling out the same blocks multiple times, but repeated
disk access is inevitable on large complex problems.&lt;/p&gt;
&lt;p&gt;This is automatic. Dask now includes an &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;inline&lt;/span&gt;&lt;/code&gt; function that does this
for you. You just give it a set of “fast” functions to ignore, e.g.&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;dsk2 = inline(dsk, [np.transpose, ndslice, add, mul, ...])
&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 132)&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="scheduler"&gt;
&lt;h1&gt;Scheduler&lt;/h1&gt;
&lt;p&gt;Now that we have a nice dask to crunch on, we run those tasks with multiple
worker threads. This is the job of a &lt;em&gt;scheduler&lt;/em&gt;.&lt;/p&gt;
&lt;p&gt;&lt;img src="/images/threads.jpg"
      align="right"
      alt="Thread pool, courtesy of sewingmantra.com"
      width="20%"&gt;&lt;/p&gt;
&lt;p&gt;We build and document such a scheduler
&lt;a class="reference external" href="https://github.com/ContinuumIO/dask/blob/master/dask/threaded.py"&gt;here&lt;/a&gt;. It
targets a shared-memory single-process multi-threaded environment. It replaces
the elegant 20 line reference solution with a large blob of ugly code
filled with locks and mutable state. Still, it manages the computation sanely,
performs some critical optimizations, and uses all of our hardware cores
(Moore’s law is dead! Long live Moore’s law!)&lt;/p&gt;
&lt;p&gt;Many NumPy operations release the GIL and so are highly amenable to
multi-threading. NumPy programs do not suffer the
single-active-core-per-process limitation of most Python code.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 154)&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="approach"&gt;
&lt;h1&gt;Approach&lt;/h1&gt;
&lt;p&gt;We follow a fairly standard model. We create a &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;ThreadPool&lt;/span&gt;&lt;/code&gt; with a fixed
number of workers. We analyze the dask to determine “ready to run” tasks.
We send a task to each of our worker threads. As they finish they update the
run-state, marking jobs as finished, inserting their results into a shared
cache, and then marking new jobs as ready based on the newly available data.
This update process is fully indexed and handled by the worker threads
themselves (with appropriate locks) making the overhead negligible and
hopefully scalable to complex workloads.&lt;/p&gt;
&lt;p&gt;When a newly available worker selects a new ready task they often have several
to choose from. We have a choice. The choice we make here is very powerful.
The next section talks about our selection policy:&lt;/p&gt;
&lt;p&gt;&lt;em&gt;Select tasks that immediately free data resources on completion&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 171)&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="optimizations"&gt;
&lt;h1&gt;Optimizations&lt;/h1&gt;
&lt;p&gt;Our policy to prefer tasks that free resources lets us run many computations in
a very small space. We now show three expressions, their resulting schedules,
and an animation showing the scheduler walk through those schedules. These
were taken from live runs.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 178)&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="example-embarrassingly-parallel-computation"&gt;
&lt;h1&gt;Example: Embarrassingly parallel computation&lt;/h1&gt;
&lt;p&gt;&lt;img src="/images/dask/embarrassing.gif"
      align="right"
      width="50%"
      alt="Embarassingly parallel dask"&gt;&lt;/p&gt;
&lt;p&gt;On the right we show an animated GIF of the progress of the following
embarrassingly parallel computation:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;expr = (((A + 1) * 2) ** 3)
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;Circles represent computations, boxes represent data.&lt;/p&gt;
&lt;p&gt;Red means actively taking up resources. Red is bad.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Red circles: tasks currently executing in a thread&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Red boxes: data currently residing in the cache occupying precious memory&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;Blue means finished or released. Blue is good.&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;Blue circles: finished tasks&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Blue boxes: data released from memory because we no longer need it for any
task&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;p&gt;We want to turn all nodes blue while minimizing the number of red boxes we have
at any given time.&lt;/p&gt;
&lt;p&gt;The policy to execute tasks that free resources causes “vertical” execution
when possible. In this example our approach is optimal because the number of
red boxes is kept small throughout the computation. We have one only red box
for each of our four threads.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 211)&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="example-more-complex-computation-with-reductions"&gt;
&lt;h1&gt;Example: More complex computation with Reductions&lt;/h1&gt;
&lt;p&gt;&lt;img src="/images/dask/normalized-b.gif"
      align="right"
      width="35%"
      alt="More complex dask"&gt;&lt;/p&gt;
&lt;p&gt;We show a more complex expression:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;expr = (B - B.mean(axis=0)) + (B.T / B.std())
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;This extends the class of expressions that we’ve seen so far to reductions and
reductions along an axis. The per-chunk reductions start at the bottom and
depend only on the chunk from which they originated. These per-chunk results
are then concatenated together and re-reduced with the large circles (zoom in
to see the text &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;concatenate&lt;/span&gt;&lt;/code&gt; in there.) The next level takes these (small)
results and the original data again (note the long edges back down the bottom
data resources) which result in per-chunk subtractions and divisions.&lt;/p&gt;
&lt;p&gt;From there on out the work is embarrassing, resembling the computation above.
In this case we have relatively little parallelism, so the frontier of red
boxes covers the entire image; fortunately the dataset is small.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 234)&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="example-fail-case"&gt;
&lt;h1&gt;Example: Fail Case&lt;/h1&gt;
&lt;p&gt;&lt;img src="/images/dask/fail-case.gif"
      align="right"
      width="50%"
      alt="A case where our scheduling algorithm fails to avoid intermediates"&gt;&lt;/p&gt;
&lt;p&gt;We show a case where our greedy solution fails miserably:&lt;/p&gt;
&lt;div class="highlight-none notranslate"&gt;&lt;div class="highlight"&gt;&lt;pre&gt;&lt;span&gt;&lt;/span&gt;expr = (A.T.dot(B) - B.mean(axis=0))
&lt;/pre&gt;&lt;/div&gt;
&lt;/div&gt;
&lt;p&gt;The two large functions on the second level from the bottom are the computation
of the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt;. These are cheap and, once completed,
allow each of the blocks of the dot product to terminate quickly and release
memory.&lt;/p&gt;
&lt;p&gt;Tragically these &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;mean&lt;/span&gt;&lt;/code&gt; computations execute at the last possible moment,
keeping all of the dot product result trapped in the cache.
We have almost a full row of red boxes at one point in the computation.&lt;/p&gt;
&lt;p&gt;In this case our greedy solution was short sighted; a slightly more global
solution would quickly select these large circles to run quickly. Perhaps
betweenness centrality would resole this problem.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 258)&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="on-disk-caching"&gt;
&lt;h1&gt;On-disk caching&lt;/h1&gt;
&lt;p&gt;We’ll never build a good enough scheduler. We need to be able to fall back
to on-disk caching. Actually this isn’t terrible. High
performance SSDs get close to 1 GB/second throughput and in the complex cases
where data-aware scheduling fails we probably compute slower than that anyway.&lt;/p&gt;
&lt;p&gt;I have a little disk-backed dictionary project,
&lt;a class="reference external" href="https://github.com/mrocklin/chest/"&gt;&lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;chest&lt;/span&gt;&lt;/code&gt;&lt;/a&gt;, for this but it’s immature. In
general I’d like to see more projects implement the &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;dict&lt;/span&gt;&lt;/code&gt; interface with
interesting policies.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 270)&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="trouble-with-binary-data-stores"&gt;
&lt;h1&gt;Trouble with binary data stores&lt;/h1&gt;
&lt;p&gt;I have a confession, the first computation, the very large dot product,
sometimes crashes my machine. While then scheduler manages memory well I have
a memory leak somewhere. I suspect that I use HDF5 improperly.&lt;/p&gt;
&lt;p&gt;I also tried doing this with &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;bcolz&lt;/span&gt;&lt;/code&gt;. Sadly nd-chunking is not well
supported. &lt;a class="reference external" href="https://groups.google.com/forum/#!topic/bcolz/6pddtrKMqQc"&gt;email thread
here&lt;/a&gt; &lt;a class="reference external" href="https://groups.google.com/forum/#!topic/bcolz/330-llmA3ps"&gt;and
here&lt;/a&gt;.&lt;/p&gt;
&lt;aside class="system-message"&gt;
&lt;p class="system-message-title"&gt;System Message: WARNING/2 (&lt;span class="docutils literal"&gt;/opt/build/repo/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 281)&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="expression-scope"&gt;
&lt;h1&gt;Expression Scope&lt;/h1&gt;
&lt;p&gt;Blaze currently generates dasks for the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Elementwise operations (like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;+&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;*&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;exp&lt;/span&gt;&lt;/code&gt;, &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;log&lt;/span&gt;&lt;/code&gt;, …)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Dimension shuffling like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.transpose&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Tensor contraction like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.tensordot&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Reductions like &lt;code class="docutils literal notranslate"&gt;&lt;span class="pre"&gt;np.mean(...,&lt;/span&gt; &lt;span class="pre"&gt;axis=...)&lt;/span&gt;&lt;/code&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;All combinations of the above&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;We also comply with the NumPy API on these operations.. At the time of writing
notable missing elements include the following:&lt;/p&gt;
&lt;ol class="arabic simple"&gt;
&lt;li&gt;&lt;p&gt;Slicing (though this should be easy to add)&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Solve, SVD, or any more complex linear algebra. There are good solutions
to this implemented in other linear algebra software (Plasma,
Flame, Elemental, …) but I’m not planning to go down that path until
lots of people start asking for it.&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;Anything that NumPy can’t do.&lt;/p&gt;&lt;/li&gt;
&lt;/ol&gt;
&lt;p&gt;I’d love to hear what’s important to the community. Re-implementing all of
NumPy is hard, re-implementing a few choice pieces of NumPy is relatively
straightforward. Knowing what those few choices pieces are requires community
involvement.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 306)&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="bigger-ideas"&gt;
&lt;h1&gt;Bigger ideas&lt;/h1&gt;
&lt;p&gt;My experience building dynamic schedulers is limited and my approach is likely
suboptimal. It would be great to see other approaches. None of the logic in
this post is specific to Blaze or even to NumPy. To build a scheduler you only
need to understand the model of a graph of computations with data dependencies.&lt;/p&gt;
&lt;p&gt;If we were ambitious we might consider a distributed scheduler to execute these
task graphs across many nodes in a distributed-memory situation (like a
cluster). This is a hard problem but it would open up a different class of
computational solutions. The Blaze code to generate the dasks would not change
; the graphs that we generate are orthogonal to our choice of scheduler.&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/2015/01/06/Towards-OOC-Scheduling.md&lt;/span&gt;, line 319)&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="help"&gt;
&lt;h1&gt;Help&lt;/h1&gt;
&lt;p&gt;I could use help with all of this, either as open source work (links below) or
for money. Continuum has funding for employees and ample interesting problems.&lt;/p&gt;
&lt;p&gt;&lt;strong&gt;Links:&lt;/strong&gt;&lt;/p&gt;
&lt;ul class="simple"&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="http://dask.readthedocs.org/en/latest/"&gt;Dask spec&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;li&gt;&lt;p&gt;&lt;a class="reference external" href="https://github.com/ContinuumIO/dask/blob/master/dask/threaded.py"&gt;Scheduler implementation (with decent narrative documentation)&lt;/a&gt;&lt;/p&gt;&lt;/li&gt;
&lt;/ul&gt;
&lt;/section&gt;
</content>
    <link href="https://blog.dask.org/2015/01/06/Towards-OOC-Scheduling/"/>
    <summary>This work is supported by Continuum Analytics
and the XDATA Program
as part of the Blaze Project</summary>
    <category term="Programming" label="Programming"/>
    <category term="Python" label="Python"/>
    <category term="dask" label="dask"/>
    <category term="scipy" label="scipy"/>
    <published>2015-01-06T00:00:00+00:00</published>
  </entry>
</feed>
